├── project ├── assembly.sbt ├── buildinfo.sbt ├── plugins.sbt └── build.properties ├── examples ├── stop.sh ├── config │ ├── sink.properties │ └── bulk-source.properties ├── setup.cql ├── data.csv └── start.sh ├── .travis.yml ├── src ├── test │ ├── resources │ │ └── log4j.properties │ └── scala │ │ └── com │ │ ├── datastax │ │ └── driver │ │ │ └── core │ │ │ └── TestUtil.scala │ │ └── tuplejump │ │ └── kafka │ │ └── connect │ │ └── cassandra │ │ ├── RouteSpec.scala │ │ ├── AbstractSpec.scala │ │ ├── ConnectorLikeSpec.scala │ │ ├── CassandraSourceSpec.scala │ │ ├── CassandraSinkSpec.scala │ │ ├── SchemaSpec.scala │ │ └── TaskConfigSpec.scala ├── main │ ├── scala-2.11 │ │ └── com │ │ │ └── tuplejump │ │ │ └── kafka │ │ │ └── connect │ │ │ └── cassandra │ │ │ └── Logging.scala │ ├── scala-2.10 │ │ └── com │ │ │ └── tuplejump │ │ │ └── kafka │ │ │ └── connect │ │ │ └── cassandra │ │ │ └── Logging.scala │ └── scala │ │ └── com │ │ └── tuplejump │ │ └── kafka │ │ └── connect │ │ └── cassandra │ │ ├── CassandraSource.scala │ │ ├── CassandraSink.scala │ │ ├── AsyncPagingSourceIterator.scala │ │ ├── CassandraSourceTask.scala │ │ ├── Topology.scala │ │ ├── CassandraSinkTask.scala │ │ ├── ConnectorLike.scala │ │ ├── package.scala │ │ ├── CassandraCluster.scala │ │ ├── ConnectorSyntax.scala │ │ └── TaskConfig.scala └── it │ ├── resources │ └── setup.cql │ └── scala │ └── com │ └── tuplejump │ └── kafka │ └── connect │ └── cassandra │ ├── CassandraSinkTaskSpec.scala │ ├── CassandraClusterSpec.scala │ └── CassandraSourceTaskSpec.scala ├── .gitignore ├── scalastyle-config.xml ├── README.md └── LICENSE /project/assembly.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.1") 2 | -------------------------------------------------------------------------------- /project/buildinfo.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.eed3si9n" % "sbt-buildinfo" % "0.5.0") 2 | -------------------------------------------------------------------------------- /examples/stop.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | if [ -f "demo.pid" ] 4 | then 5 | kill -9 $(cat demo.pid) 6 | rm demo.pid 7 | fi -------------------------------------------------------------------------------- /examples/config/sink.properties: -------------------------------------------------------------------------------- 1 | name=cassandra-sink-connector 2 | connector.class=com.tuplejump.kafka.connect.cassandra.CassandraSink 3 | tasks.max=1 4 | topics=demo 5 | cassandra.sink.route.demo=demo.event_store_sink -------------------------------------------------------------------------------- /examples/config/bulk-source.properties: -------------------------------------------------------------------------------- 1 | name=cassandra-source-bulk 2 | connector.class=com.tuplejump.kafka.connect.cassandra.CassandraSource 3 | topic=demo 4 | cassandra.source.route.demo=select * from demo.event_store 5 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | 3 | jdk: 4 | - oraclejdk8 5 | 6 | scala: 7 | - 2.10.6 8 | - 2.11.7 9 | 10 | # Helps it:test with Cassandra plugin run clean 11 | # sudo: true 12 | 13 | script: 14 | - sbt ++$TRAVIS_SCALA_VERSION test it:compile 15 | -------------------------------------------------------------------------------- /src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=INFO, stdout 2 | 3 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 4 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n 6 | 7 | log4j.logger.com.tuplejump.kafka.connect.cassandra=DEBUG 8 | log4j.logger.org.apache.kafka=ERROR -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | resolvers += "Sonatype OSS Releases" at "https://oss.sonatype.org/content/repositories/releases" 2 | 3 | addSbtPlugin("com.tuplejump.com.github.hochgi" % "sbt-cassandra" % "1.0.4") 4 | addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.8.2") 5 | addSbtPlugin("de.heikoseeberger" % "sbt-header" % "1.5.0") 6 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.8.0") -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.log 3 | 4 | # sbt specific 5 | .cache 6 | .history 7 | .lib/ 8 | dist/* 9 | target/ 10 | lib_managed/ 11 | src_managed/ 12 | project/boot/ 13 | project/plugins/project/ 14 | 15 | # Scala-IDE specific 16 | .scala_dependencies 17 | .worksheet 18 | 19 | .idea/ 20 | 21 | examples/cassandra/ 22 | examples/setup.cql 23 | examples/kafka_2.11-0.9.0.1/ 24 | examples/cassandraLog 25 | examples/zkLog 26 | examples/serverLog 27 | examples/connectLog 28 | -------------------------------------------------------------------------------- /examples/setup.cql: -------------------------------------------------------------------------------- 1 | CREATE KEYSPACE IF NOT EXISTS demo WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; 2 | 3 | USE demo; 4 | 5 | CREATE TABLE IF NOT EXISTS demo.event_store( 6 | app_id text, 7 | event_type text, 8 | subscription_type text, 9 | event_ts timestamp, 10 | PRIMARY KEY((app_id, event_type), event_ts) 11 | ); 12 | 13 | CREATE TABLE IF NOT EXISTS demo.event_store_sink( 14 | app_id text, 15 | event_type text, 16 | subscription_type text, 17 | event_ts timestamp, 18 | PRIMARY KEY((app_id, event_type), event_ts) 19 | ); 20 | 21 | COPY event_store(app_id,subscription_type,event_type,event_ts) FROM 'data.csv' ; 22 | 23 | 24 | -------------------------------------------------------------------------------- /src/main/scala-2.11/com/tuplejump/kafka/connect/cassandra/Logging.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import com.typesafe.scalalogging.StrictLogging 20 | 21 | trait Logging extends StrictLogging 22 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to Tuplejump Software Pvt. Ltd. under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. Tuplejump Software Pvt. Ltd. licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | # 18 | # 19 | 20 | sbt.version = 0.13.11 -------------------------------------------------------------------------------- /src/main/scala-2.10/com/tuplejump/kafka/connect/cassandra/Logging.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import org.slf4j.LoggerFactory 20 | 21 | trait Logging { 22 | 23 | protected val logger = LoggerFactory.getLogger(getClass.getName) 24 | 25 | } 26 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/CassandraSource.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import org.apache.kafka.connect.connector.Task 20 | import org.apache.kafka.connect.source.SourceConnector 21 | 22 | /** Cassandra [[SourceConnector]] data flow: a Cassandra 23 | * source with a Kafka sink. 24 | */ 25 | class CassandraSource extends SourceConnector with CassandraConnector { 26 | 27 | override val taskClass: Class[_ <: Task] = classOf[CassandraSourceTask] 28 | 29 | } 30 | -------------------------------------------------------------------------------- /src/test/scala/com/datastax/driver/core/TestUtil.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.datastax.driver.core 18 | 19 | import com.datastax.driver.core.ColumnDefinitions.Definition 20 | 21 | /** 22 | * This is a hack to create a dummy ColumnDefinition for unit-tests. 23 | * Could not mock using Mockito since the constructors are not public 24 | */ 25 | object TestUtil { 26 | 27 | def getColumnDef(cols:Map[String,DataType]):ColumnDefinitions = { 28 | val definitions: Array[Definition] = cols.map { 29 | case (x, y) => new Definition("test", "dummy", x, y) 30 | }.toArray 31 | new ColumnDefinitions(definitions, CodecRegistry.DEFAULT_INSTANCE) 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/CassandraSink.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import org.apache.kafka.connect.connector.{ConnectorContext, Task} 20 | import org.apache.kafka.connect.sink.SinkConnector 21 | 22 | /** Cassandra [[SinkConnector]] data flow: a Kafka source 23 | * to a Cassandra sink. 24 | */ 25 | class CassandraSink extends SinkConnector with CassandraConnector { 26 | 27 | override val taskClass: Class[_ <: Task] = classOf[CassandraSinkTask] 28 | 29 | /** Roadmap: waiting for CDC so we don't have to use triggers. */ 30 | override def initialize(ctx: ConnectorContext): Unit = () 31 | 32 | } 33 | -------------------------------------------------------------------------------- /examples/data.csv: -------------------------------------------------------------------------------- 1 | website,annual,purchase,1457713158000 2 | iphone,monthly,others,1457713159000 3 | android,trial,purchase,1457713160000 4 | website,monthly,renewal,1457713161000 5 | website,monthly,others,1457713162000 6 | android,monthly,feedback,1457713163000 7 | iphone,annual,enquiry,1457713164000 8 | iphone,monthly,purchase,1457713165000 9 | website,trial,others,1457713166000 10 | android,none,feedback,1457713167000 11 | android,annual,renewal,1457713168000 12 | iphone,none,others,1457713169000 13 | android,annual,purchase,1457713170000 14 | website,trial,renewal,1457713171000 15 | iphone,none,others,1457713172000 16 | android,trial,purchase,1457713173000 17 | iphone,annual,others,1457713174000 18 | android,monthly,renewal,1457713175000 19 | iphone,monthly,renewal,1457713176000 20 | website,trial,feedback,1457713177000 21 | iphone,monthly,renewal,1457713321000 22 | android,annual,feedback,1457713322000 23 | android,none,feedback,1457713323000 24 | website,none,enquiry,1457713324000 25 | website,none,feedback,1457713325000 26 | website,monthly,enquiry,1457713326000 27 | website,trial,enquiry,1457713327000 28 | android,none,renewal,1457713328000 29 | website,monthly,purchase,1457713329000 30 | iphone,annual,purchase,1457713330000 31 | website,annual,others,1457713331000 32 | website,monthly,enquiry,1457713332000 33 | android,trial,purchase,1457713333000 34 | iphone,none,enquiry,1457713334000 35 | iphone,annual,enquiry,1457713335000 36 | website,none,feedback,1457713336000 37 | iphone,trial,enquiry,1457713337000 38 | website,monthly,renewal,1457713338000 39 | android,none,enquiry,1457713339000 40 | android,monthly,purchase,1457713340000 41 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/AsyncPagingSourceIterator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import com.datastax.driver.core.{Row, ResultSet} 20 | 21 | /** For iterating over a large, paged ResultSet more efficiently, asynchronously 22 | * pre-fetching the next `page`. 23 | * 24 | * @param results result set returned from the driver 25 | * @param pageSize if there are less than this rows available without blocking, 26 | * initiates fetching the next page 27 | */ 28 | class AsyncPagingSourceIterator(results: ResultSet, 29 | pageSize: Int, 30 | limit: Option[Long] = None) extends Iterator[Row] { 31 | 32 | private var _read = 0 33 | 34 | private[this] val iterator = results.iterator 35 | 36 | /** Returns the number of successfully read elements. */ 37 | def read: Int = _read 38 | 39 | def done: Boolean = read == results.getAvailableWithoutFetching 40 | 41 | override def hasNext: Boolean = limit 42 | .map(max => _read < max && iterator.hasNext) 43 | .getOrElse(iterator.hasNext) 44 | 45 | /** Internal fetch attempt is async. */ 46 | override def next(): Row = { 47 | if (hasNextWindow) results.fetchMoreResults() 48 | 49 | val n = iterator.next() 50 | _read += 1 51 | n 52 | } 53 | 54 | /** If `!isFullyFetched` alone does not guarantee the result set is not 55 | * exhausted. This verifies that it is. */ 56 | private def hasNextWindow: Boolean = 57 | !results.isFullyFetched && !results.isExhausted && 58 | results.getAvailableWithoutFetching < pageSize 59 | } 60 | -------------------------------------------------------------------------------- /src/it/resources/setup.cql: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2016 Tuplejump 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | /* Always clean to not run against existing data. */ 18 | DROP KEYSPACE IF EXISTS githubstats; 19 | 20 | DROP KEYSPACE IF EXISTS test; 21 | 22 | CREATE KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1 }; 23 | 24 | CREATE TABLE IF NOT EXISTS test.kv ( 25 | key text, 26 | value int, 27 | PRIMARY KEY (key)); 28 | 29 | CREATE TABLE test.playlists ( 30 | id bigint, 31 | song_order int, 32 | song_id bigint, 33 | title text, 34 | album text, 35 | artist text, 36 | PRIMARY KEY (id, song_order ) ); 37 | 38 | INSERT INTO test.playlists (id, song_order, song_id, title, artist, album) 39 | VALUES (1, 4, 66, 'Ojo Rojo', 'Fu Manchu', 'No One Rides for Free'); 40 | 41 | INSERT INTO test.playlists (id, song_order, song_id, title, artist, album) 42 | VALUES (2, 1, 44, 'La Grange', 'ZZ Top', 'Tres Hombres'); 43 | 44 | INSERT INTO test.playlists (id, song_order, song_id, title, artist, album) 45 | VALUES (3, 2, 469, 'Moving in Stereo', 'Fu Manchu', 'We Must Obey'); 46 | 47 | INSERT INTO test.playlists (id, song_order, song_id, title, artist, album) 48 | VALUES (4, 3, 49077, 'Outside Woman Blues', 'Back Door Slam', 'Roll Away'); 49 | 50 | CREATE KEYSPACE githubstats WITH REPLICATION = {'class': 'SimpleStrategy', 'replication_factor': 1 }; 51 | 52 | CREATE TABLE githubstats.monthly_commits ( 53 | user VARCHAR, 54 | commits INT, 55 | month INT, 56 | year INT, 57 | PRIMARY KEY ((user), year, month) 58 | ) WITH CLUSTERING ORDER BY (year DESC, month DESC); 59 | 60 | CREATE TABLE IF NOT EXISTS test.event_store( 61 | app_id text, 62 | event_type text, 63 | subscription_type text, 64 | event_ts timestamp, 65 | PRIMARY KEY((app_id, event_type), event_ts) 66 | ); -------------------------------------------------------------------------------- /examples/start.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" 4 | 5 | waitForStart(){ 6 | until grep -q "$2" $3 7 | do 8 | echo "waiting for "$1" to start" 9 | sleep 5 10 | done 11 | } 12 | 13 | if [ ! -f "$KAFKA_HOME" ] 14 | then 15 | if [ ! -f kafka.tgz ] 16 | then 17 | echo "KAFKA_HOME is not configured. Downloading Kafka." 18 | wget "http://archive.apache.org/dist/kafka/0.9.0.1/kafka_2.11-0.9.0.1.tgz" -O kafka.tgz 19 | fi 20 | tar -xzf kafka.tgz 21 | KAFKA_HOME=$DIR/$(tar tfz kafka.tgz --exclude '*/*') 22 | fi 23 | 24 | if [ ! -f "$CASSANDRA_HOME" ] 25 | then 26 | if [ ! -f cassandra.tar.gz ] 27 | then 28 | echo "CASSANDRA_HOME is not configured. Downloading Cassandra." 29 | wget "http://archive.apache.org/dist/cassandra/3.1.1/apache-cassandra-3.1.1-bin.tar.gz" -O cassandra.tar.gz 30 | fi 31 | mkdir -p cassandra 32 | tar -xzf cassandra.tar.gz -C cassandra --strip-components=1 33 | CASSANDRA_HOME=cassandra 34 | fi 35 | 36 | if [ ! -f "kafka-connect-cassandra-assembly-0.0.7.jar" ] 37 | then 38 | echo "didnt find jar. Downloading" 39 | wget "http://downloads.tuplejump.com/kafka-connect-cassandra-assembly-0.0.7.jar" 40 | fi 41 | 42 | cp kafka-connect-cassandra-assembly-0.0.7.jar ${KAFKA_HOME}libs/ 43 | 44 | ##update path of data file 45 | sed -i "s;'data.csv';'${DIR}/data.csv';" "${DIR}/setup.cql" 46 | 47 | ##start cassandra 48 | cd ${CASSANDRA_HOME} 49 | 50 | bin/cassandra -p ${DIR}/'demo.pid' > ${DIR}/cassandraLog 51 | 52 | waitForStart "cassandra" "state jump to NORMAL" "${DIR}/cassandraLog" 53 | 54 | ##setup schema 55 | bin/cqlsh -f "${DIR}/setup.cql" 56 | 57 | cd ${KAFKA_HOME} 58 | 59 | ## start zookeeper 60 | bin/zookeeper-server-start.sh config/zookeeper.properties > ${DIR}/zkLog & 61 | ZK_PID=$! 62 | 63 | waitForStart "zookeeper" "binding to port" "${DIR}/zkLog" 64 | echo -n " "${ZK_PID} >> ${DIR}/demo.pid 65 | 66 | ## start Kafka server 67 | bin/kafka-server-start.sh config/server.properties > ${DIR}/serverLog & 68 | KAFKA_PID=$! 69 | 70 | waitForStart "kafka server" "Awaiting socket connections" "${DIR}/serverLog" 71 | echo -n " "${KAFKA_PID} >> ${DIR}/demo.pid 72 | 73 | # create topic 74 | bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic demo 75 | 76 | bin/connect-standalone.sh config/connect-standalone.properties ${DIR}/config/bulk-source.properties ${DIR}/config/sink.properties > ${DIR}/connectLog & 77 | CONNECT_PID=$! 78 | 79 | echo -n " "${CONNECT_PID} >> ${DIR}/demo.pid -------------------------------------------------------------------------------- /src/it/scala/com/tuplejump/kafka/connect/cassandra/CassandraSinkTaskSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to Tuplejump Software Pvt. Ltd. under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. Tuplejump Software Pvt. Ltd. licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | * 18 | */ 19 | package com.tuplejump.kafka.connect.cassandra 20 | 21 | import scala.collection.JavaConverters._ 22 | import org.apache.kafka.connect.data.{Schema, SchemaBuilder, Struct} 23 | import org.apache.kafka.connect.sink.{SinkRecord, SinkTaskContext} 24 | 25 | class CassandraSinkTaskSpec extends AbstractFlatSpec { 26 | 27 | val topicName = "test_kv_topic" 28 | val tableName = "test.kv" 29 | val config = sinkProperties(Map(topicName -> tableName)) 30 | 31 | it should "start sink task" in { 32 | val sinkTask = new CassandraSinkTask() 33 | val mockContext = mock[SinkTaskContext] 34 | 35 | sinkTask.initialize(mockContext) 36 | sinkTask.start(config.asJava) 37 | sinkTask.stop() 38 | } 39 | 40 | it should "save records in cassandra" in { 41 | val sinkTask = new CassandraSinkTask() 42 | val mockContext = mock[SinkTaskContext] 43 | 44 | sinkTask.initialize(mockContext) 45 | sinkTask.start(config.asJava) 46 | 47 | val valueSchema = SchemaBuilder.struct.name("record").version(1) 48 | .field("key", Schema.STRING_SCHEMA) 49 | .field("value", Schema.INT32_SCHEMA).build 50 | val value1 = new Struct(valueSchema).put("key", "pqr").put("value", 15) 51 | val value2 = new Struct(valueSchema).put("key", "abc").put("value", 17) 52 | 53 | val record1 = new SinkRecord(topicName, 1, SchemaBuilder.struct.build, "key", valueSchema, value1, 0) 54 | val record2 = new SinkRecord(topicName, 1, SchemaBuilder.struct.build, "key", valueSchema, value2, 0) 55 | 56 | sinkTask.put(List(record1, record2).asJavaCollection) 57 | 58 | sinkTask.stop() 59 | 60 | val cc = CassandraCluster.local 61 | val session = cc.session 62 | val result = session.execute(s"select count(1) from $tableName").one() 63 | val rowCount = result.getLong(0) 64 | rowCount should be(2) 65 | cc.shutdown() 66 | } 67 | } 68 | 69 | -------------------------------------------------------------------------------- /src/it/scala/com/tuplejump/kafka/connect/cassandra/CassandraClusterSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | class CassandraClusterSpec extends AbstractSpec { 20 | 21 | private val inserts = 1000 22 | 23 | "CassandraCluster" must { 24 | "configure a cluster with default settings" in { 25 | val cluster = CassandraCluster.local 26 | val session = cluster.session 27 | session.isClosed should be(false) 28 | cluster.shutdown() 29 | } 30 | "configure a cluster with valid values and handle invalid host failover" in { 31 | val config = Map(CassandraCluster.ConnectionHosts -> "127.0.0.1") 32 | val cluster = CassandraCluster(config) 33 | cluster.seedNodes.size should be(1) 34 | val session = cluster.session 35 | cluster.shutdown() 36 | } 37 | "handle whitespace in config" in { 38 | val cluster = CassandraCluster(Map(CassandraCluster.ConnectionHosts -> " 10.2.2.1, 127.0.0.1 ")) 39 | cluster.seedNodes.size should be(2) 40 | val session = cluster.session 41 | session.isClosed should be(false) 42 | cluster.shutdown() 43 | } 44 | "create, use and destroy a session" in { 45 | 46 | val namespace = "githubstats.monthly_commits" 47 | val users = Map("helena" -> 1000, "shiti" -> 1000, "velvia" -> 800, "milliondreams" -> 2000) 48 | val columnNames = List("user","commits","year","month") 49 | val columns = columnNames.mkString(",") 50 | 51 | val statements = for { 52 | (user, count) <- users 53 | month <- 1 to 12 54 | commits <- (500 to count*month).toList 55 | } yield s"INSERT INTO $namespace($columns) values('$user',$commits,2016,$month)" 56 | 57 | val cluster = CassandraCluster.local 58 | 59 | try { 60 | val session = cluster.session 61 | statements foreach (q => session.execute(q)) 62 | 63 | import scala.collection.JavaConverters._ 64 | 65 | users foreach { case (usr, commits) => 66 | val query = s"SELECT * FROM $namespace WHERE user = '$usr'" 67 | val results = session.execute(query).all().asScala 68 | results.size should be (12)//months 69 | } 70 | } finally { 71 | cluster.shutdown() 72 | } 73 | } 74 | } 75 | } 76 | 77 | -------------------------------------------------------------------------------- /src/test/scala/com/tuplejump/kafka/connect/cassandra/RouteSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import org.apache.kafka.common.config.ConfigException 20 | 21 | class RouteSpec extends AbstractSpec { 22 | 23 | import InternalConfig._ 24 | 25 | private val topics = List( 26 | "topic1", "devices", "test", "FuTopic", "_topic", "some-topic" 27 | ) 28 | 29 | "A Route" must { 30 | "not create routes given invalid configuration" in { 31 | an[ConfigException] should be thrownBy { 32 | Route(TaskConfig.SinkRoute + "sensors.stream", "timeseries.") 33 | } 34 | 35 | List( 36 | Route("", "timeseries.sensors"), 37 | Route(TaskConfig.SourceRoute + "sensors.stream", "select a,b,c from x"), 38 | Route(TaskConfig.SourceRoute + "sensors.stream", "select * from x"), 39 | Route(TaskConfig.SourceRoute + "sensors.stream", "select a,b,c from"), 40 | Route(TaskConfig.SourceRoute + "topic", "select a,b,c from x."), 41 | Route(TaskConfig.SourceRoute + "topic", "select a,b,c from .y"), 42 | Route(TaskConfig.SourceRoute + "", ""), 43 | Route(TaskConfig.SourceRoute + "topic", "SELECT a,b,c FROM "), 44 | Route(TaskConfig.SourceRoute + "topic", "SELECT a,b,c FROM"), 45 | Route(TaskConfig.SourceRoute + "topic", "SELECT * ") 46 | ).flatten.isEmpty should be (true) 47 | 48 | List( 49 | Route(TaskConfig.SinkRoute + "sensors.stream", "a"), 50 | Route(TaskConfig.SinkRoute + "t", "ab"), 51 | Route(TaskConfig.SinkRoute + "topic", "") 52 | ).flatten.isEmpty should be (true) 53 | } 54 | 55 | "create routes from valid source route config" in { 56 | (for { 57 | topic <- topics 58 | query <- List( 59 | s"SELECT * FROM a.$topic WHERE token(k) > token(42)", 60 | s"select a from timeseries.$topic") 61 | } yield Route(TaskConfig.SourceRoute + topic, query)) 62 | .flatten.size should be (topics.size * 2) 63 | } 64 | "create routes from valid sink route config" in { 65 | (for { 66 | topic <- topics 67 | ns <- List(s"keyspace.$topic", "a.b", "a_x.b_x") 68 | } yield Route(TaskConfig.SinkRoute + topic, ns)) 69 | .flatten.size should be (topics.size * 3) 70 | } 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /src/test/scala/com/tuplejump/kafka/connect/cassandra/AbstractSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import org.scalatest.mock.MockitoSugar 20 | import org.scalatest.{BeforeAndAfterAll, WordSpec, FlatSpec, Matchers} 21 | 22 | trait ConfigFixture extends { 23 | import InternalConfig._,Types._,TaskConfig._ 24 | 25 | final val EmptyProperties = Map.empty[String, String] 26 | 27 | protected val multipleTopics: String = "test1,test2" 28 | 29 | protected lazy val topicList = multipleTopics.split(TaskConfig.TopicSeparator).toList 30 | 31 | protected lazy val sourceSchemas: List[Schema] = 32 | topicList.zipWithIndex.map { case (t,i) => 33 | val route = Route(TaskConfig.SourceRoute + t, s"SELECT * FROM ks$i.table$i").get 34 | Schema(route, Nil, Nil, Nil, List("a", "b"), "") 35 | } 36 | 37 | protected lazy val sinkSchemas: List[Schema] = 38 | topicList.zipWithIndex.map { case (t,i) => 39 | val route = Route(TaskConfig.SinkRoute + t, s"ks$i.table$i").get 40 | Schema(route, Nil, Nil, Nil, List("a", "b"), "") 41 | } 42 | 43 | protected lazy val sinkTopicMap = sinkSchemas.map(s => s.route.topic -> s.namespace).toMap 44 | 45 | protected lazy val commonConfig = Map( 46 | CassandraCluster.ConnectionHosts -> CassandraCluster.DefaultHosts, 47 | CassandraCluster.ConnectionPort -> CassandraCluster.DefaultPort.toString, 48 | CassandraCluster.ConnectionConsistency -> CassandraCluster.DefaultConsistency.name 49 | ) 50 | 51 | protected def sinkProperties(config:Map[String,String] = sinkTopicMap): Map[String, String] = 52 | config.map { case (topic,ns) => TaskConfig.SinkRoute + topic -> ns} ++ commonConfig 53 | 54 | protected def sourceProperties(query: String, topic: String): Map[String,String] = 55 | commonConfig ++ Map(TaskConfig.SourceRoute + topic -> query) 56 | 57 | protected def sinkConfig(topic: TopicName, 58 | keyspace: KeyspaceName, 59 | table: TableName, 60 | columnNames: List[ColumnName] = Nil): SinkConfig = { 61 | import com.tuplejump.kafka.connect.cassandra.Syntax.PreparedQuery 62 | 63 | val route = Route(TaskConfig.SinkRoute + topic, s"$keyspace.$table").get 64 | val schema = Schema(route, Nil, Nil, Nil, columnNames, "") 65 | SinkConfig(schema, PreparedQuery(schema), WriteOptions(DefaultSinkConsistency)) 66 | } 67 | } 68 | 69 | trait AbstractSpec extends WordSpec with Matchers with BeforeAndAfterAll with ConfigFixture 70 | 71 | trait AbstractFlatSpec extends FlatSpec with Matchers with BeforeAndAfterAll 72 | with ConfigFixture with MockitoSugar 73 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/CassandraSourceTask.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import java.util.{List => JList, Map => JMap, ArrayList => JArrayList} 20 | 21 | import org.apache.kafka.connect.connector.Task 22 | import org.apache.kafka.connect.source.{SourceRecord, SourceTask} 23 | 24 | /** A Cassandra `SourceTask` run by a Kafka `WorkerSourceTask`. 25 | * In a Cassandra source task, the query may be selected column reads 26 | * of pre-aggregated data vs reflect all columns in the table schema. 27 | */ 28 | class CassandraSourceTask extends SourceTask with CassandraTask { 29 | import InternalConfig._ 30 | 31 | //until CDC, only for ranges. not using yet, coming next PR. 32 | private var checkpoint: Option[Any] = None 33 | 34 | private var partitions = new JArrayList[JMap[String, String]]() 35 | 36 | protected final val taskClass: Class[_ <: Task] = classOf[CassandraSourceTask] 37 | 38 | override def start(taskConfig: JMap[String, String]): Unit = { 39 | super.start(taskConfig) 40 | //TODO context.offsetStorageReader.offsets(partitions) 41 | } 42 | 43 | /** Returns a list of records when available by polling this SourceTask 44 | * for new records. From the kafka doc: "This method should block if 45 | * no data is currently available." 46 | * 47 | * Initial implementation only supports bulk load for a query. 48 | */ 49 | override def poll: JList[SourceRecord] = { 50 | 51 | val records = new JArrayList[SourceRecord]() 52 | val offset = EmptyJMap//context.offsetStorageReader.offset(EmptyJMap) //TODO 53 | val partition = EmptyJMap //TODO 54 | 55 | for { 56 | sc <- taskConfig.source 57 | iterator <- page(sc) 58 | row <- iterator 59 | } { 60 | val record = row.as(sc.schema.route.topic, partition, offset) 61 | records.add(record) 62 | if (iterator.done) checkpoint = None //TODO 63 | record 64 | } 65 | 66 | records 67 | } 68 | 69 | private def page(sc: SourceConfig): Option[AsyncPagingSourceIterator] = { 70 | //TODO need CDC: https://github.com/tuplejump/kafka-connector/issues/9 71 | val query = sc.query match { 72 | case q if q.hasPatternT => 73 | //TODO remove Thread.sleep with better option like timestamp.fromNow...etc 74 | Thread.sleep(sc.query.pollInterval) 75 | sc.query.slide 76 | case q => 77 | // TODO typed: https://tuplejump.atlassian.net/browse/DB-56 timeuuid,timestamp... 78 | // by type: WHERE {columnToMove} > checkpoint.value with columnType 79 | sc.query 80 | } 81 | 82 | val rs = session.execute(query.cql) 83 | if (rs.getAvailableWithoutFetching > 0) Some(new AsyncPagingSourceIterator(rs, sc.options.fetchSize)) 84 | else None 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/Topology.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import java.net.{InetSocketAddress, InetAddress} 20 | 21 | import scala.collection.JavaConverters._ 22 | 23 | /** A WIP. 24 | * INTERNAL API. 25 | */ 26 | private[cassandra] trait Topology extends CassandraClusterApi { 27 | import Syntax._ 28 | 29 | /** Maps rpc addresses to listen addresses for every cluster node. 30 | * If rpc address is not known, returns the same address. */ 31 | lazy val rpcToListenAddress: InetAddress => InetAddress = 32 | (for { 33 | row <- localNode(RpcAddressColumn, ListenAddressColumn).all.asScala 34 | rpcAddress <- Option(row.getInet(RpcAddressColumn)) 35 | listenAddress = row.getInet(ListenAddressColumn) 36 | } yield (rpcAddress, listenAddress)).toMap.withDefault(identity) 37 | 38 | /** 39 | * @param endpoints which nodes the data partition is located on 40 | */ 41 | def preferredNodes(endpoints: Iterable[InetAddress]): Iterable[String] = 42 | endpoints flatMap hostNames 43 | 44 | private def hostNames(address: InetAddress): Set[String] = 45 | Set(address, rpcToListenAddress(address)) 46 | .flatMap(a => Set(a.getHostAddress, a.getHostName)) 47 | } 48 | 49 | trait PartitionProvider 50 | trait OffsetProvider 51 | 52 | // Should return a set of hostnames or IP addresses 53 | // describing the preferred hosts for that scan split 54 | /** INTERNAL API. */ 55 | private[cassandra] final case class CassandraTokenRangeSplit(startToken: String, 56 | endToken: String, 57 | replicas: Set[InetSocketAddress]) { 58 | 59 | def hostnames: Set[String] = replicas.flatMap { r => 60 | Set(r.getHostString, r.getAddress.getHostAddress) 61 | } 62 | } 63 | 64 | /** INTERNAL API. */ 65 | private[cassandra] final case class Token(value: BigInt) extends Ordered[BigInt] { 66 | override def compare(that: BigInt): Int = value.compare(that) 67 | override def toString: String = value.toString 68 | } 69 | 70 | /** INTERNAL API. */ 71 | private[cassandra] final case class TokenRange(start: Token, 72 | end: Token, 73 | replicas: Set[InetAddress], 74 | size: Long) 75 | 76 | /** The partitioner is responsible for distributing groups of rows (by 77 | * partition key) across nodes in the cluster. 78 | * 79 | * Besides org.apache.cassandra.dht.Murmur3Partitioner, partitioners 80 | * included for backwards compatibility include: 81 | * RandomPartitioner, ByteOrderedPartitioner, and OrderPreservingPartitioner. 82 | * 83 | * INTERNAL API. 84 | */ 85 | private[cassandra] final case class CassandraPartition(index: Int, 86 | addresses: Iterable[InetAddress], 87 | ranges: Iterable[TokenRange], 88 | rowCount: Long) 89 | -------------------------------------------------------------------------------- /src/test/scala/com/tuplejump/kafka/connect/cassandra/ConnectorLikeSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import com.tuplejump.kafka.connect.cassandra.TaskConfig.TaskStrategy.OneToMany 20 | import org.apache.kafka.connect.errors.ConnectException 21 | 22 | abstract class ConnectorLikeSpec extends AbstractSpec with ConnectorLike 23 | 24 | class EmptyConnectorLikeSpec extends ConnectorLikeSpec { 25 | "ConnectorLike" must { 26 | "fail on empty source task config" in { 27 | an[ConnectException] should be thrownBy { 28 | configure(Map.empty[String, String], classOf[CassandraSourceTask]) 29 | } 30 | } 31 | "fail on empty sink task config" in { 32 | an[ConnectException] should be thrownBy { 33 | configure(Map.empty[String, String], classOf[CassandraSinkTask]) 34 | } 35 | } 36 | "fail on source task config with no route mappings" in { 37 | an[ConnectException] should be thrownBy { 38 | configure(commonConfig, classOf[CassandraSourceTask]) 39 | } 40 | } 41 | "fail on sink task config with no route mappings" in { 42 | an[ConnectException] should be thrownBy { 43 | configure(commonConfig, classOf[CassandraSinkTask]) 44 | } 45 | } 46 | } 47 | } 48 | 49 | class SourceConnectorLikeSpec extends ConnectorLikeSpec { 50 | "A Source ConnectorLike" must { 51 | "validate configuration on startup" in { 52 | val query = "SELECT * FROM music.playlists" 53 | val topic = "playlists" 54 | configure(sourceProperties(query, topic), source) 55 | configT.nonEmpty should be (true) 56 | configT.keySet.exists(_.contains("cassandra.connection")) should be (true) 57 | configT.keySet.exists(_.contains("cassandra.source")) should be (true) 58 | configT.keySet.exists(_.contains("cassandra.sink")) should be (false) 59 | 60 | taskStrategy should === (TaskConfig.TaskStrategy.OneToOne) 61 | routes.size should be (1) 62 | val route = routes.head 63 | route.topic should ===("playlists") 64 | route.keyspace should be ("music") 65 | route.table should be ("playlists") 66 | route.query should be (Some(query)) 67 | } 68 | } 69 | } 70 | 71 | class SinkConnectorLikeSpec extends AbstractSpec with ConnectorLike { 72 | 73 | protected def withOneToMany(config:Map[String,String]): Map[String,String] = 74 | commonConfig ++ config ++ Map(TaskConfig.TaskParallelismStrategy -> OneToMany.key) 75 | 76 | "A Sink ConnectorLike" must { 77 | "validate configuration on startup" in { 78 | val props = withOneToMany(sinkProperties(sinkTopicMap)) 79 | configure(props, sink) 80 | configT.nonEmpty should be (true) 81 | configT.keySet.exists(_.contains("cassandra.connection")) should be (true) 82 | configT.keySet.exists(_.contains("cassandra.task")) should be (true) 83 | configT.keySet.exists(_.contains("cassandra.sink")) should be (true) 84 | configT.keySet.exists(_.contains("cassandra.source")) should be (false) 85 | 86 | routes.size should be (2) 87 | taskStrategy should ===(TaskConfig.TaskStrategy.OneToMany) 88 | } 89 | } 90 | } -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/CassandraSinkTask.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import java.util.{Collection => JCollection, Map => JMap} 20 | 21 | import scala.util.Try 22 | import scala.util.control.NonFatal 23 | import scala.collection.JavaConverters._ 24 | import org.apache.kafka.clients.consumer.OffsetAndMetadata 25 | import org.apache.kafka.common.TopicPartition 26 | import org.apache.kafka.connect.sink.{SinkRecord, SinkTask} 27 | import org.apache.kafka.connect.connector.Task 28 | import org.apache.kafka.connect.errors.ConnectException 29 | import com.datastax.driver.core.{PreparedStatement, Session} 30 | 31 | /** A Cassandra `SinkTask` run by a Kafka `WorkerSinkTask`. */ 32 | class CassandraSinkTask extends SinkTask with CassandraTask { 33 | import InternalConfig._ 34 | 35 | protected val taskClass: Class[_ <: Task] = classOf[CassandraSinkTask] 36 | 37 | override def start(taskConfig: JMap[String,String]): Unit = { 38 | super.start(taskConfig) 39 | /* TODO for (tp <- context.assignment.asScala) */ 40 | } 41 | 42 | /** Writes records from Kafka to Cassandra asynchronously and non-blocking. */ 43 | override def put(records: JCollection[SinkRecord]): Unit = { 44 | // ensure only those topic schemas configured are attempted to store in C* 45 | // TODO handle reconfigure 46 | for (sc <- taskConfig.sink) { 47 | val byTopic = records.asScala.filter(_.topic == sc.schema.route.topic) 48 | write(sc, byTopic) 49 | } 50 | } 51 | 52 | private def write(sc: SinkConfig, byTopic: Iterable[SinkRecord]): Unit = { 53 | // TODO needs ticket: if (byTopic.size > 1) boundWrite(sc, byTopic) else 54 | for (record <- byTopic) { 55 | val query = record.as(sc.schema.namespace) 56 | Try(session.executeAsync(query.cql)) recover { case NonFatal(e) => 57 | throw new ConnectException( 58 | s"Error executing ${byTopic.size} records for schema '${sc.schema}'.", e) 59 | } 60 | } 61 | } 62 | 63 | // for queries that are executed multiple times, one topic per keyspace.table 64 | private def boundWrite(sc: SinkConfig, byTopic: Iterable[SinkRecord]): Unit = { 65 | val statement = prepare(session, sc) 66 | val futures = for (record <- byTopic) yield { 67 | val query = record.as(sc.schema.namespace) 68 | try { 69 | val bs = statement.bind(query.cql) 70 | session.executeAsync(bs) 71 | } catch { case NonFatal(e) => 72 | throw new ConnectException( 73 | s"Error executing ${byTopic.size} records for schema '${sc.schema}'.", e) 74 | } 75 | } 76 | 77 | // completed before exiting thread. 78 | for (rs <- futures) rs.getUninterruptibly 79 | } 80 | 81 | private def prepare(session: Session, sc: SinkConfig): PreparedStatement = 82 | try session.prepare(sc.query.cql).setConsistencyLevel(sc.options.consistency) catch { 83 | case NonFatal(e) => throw new ConnectException( 84 | s"Unable to prepare statement ${sc.query.cql}: ${e.getMessage}", e) 85 | } 86 | 87 | /** This method is not relevant as we insert every received record in Cassandra. */ 88 | override def flush(offsets: JMap[TopicPartition, OffsetAndMetadata]): Unit = () 89 | 90 | } 91 | -------------------------------------------------------------------------------- /src/test/scala/com/tuplejump/kafka/connect/cassandra/CassandraSourceSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import scala.collection.JavaConverters._ 20 | import org.apache.kafka.connect.errors.ConnectException 21 | import com.tuplejump.kafka.connect.cassandra.TaskConfig.TaskStrategy 22 | 23 | class CassandraSourceSpec extends AbstractFlatSpec { 24 | 25 | private def config(taskStrategy: String) = (commonConfig ++ Map( 26 | TaskConfig.TaskParallelismStrategy -> taskStrategy, 27 | TaskConfig.SourceRoute + "playlists1" -> "Select * from music.playlists1", 28 | TaskConfig.SourceRoute + "playlists2" -> "Select * from music2.playlists2", 29 | TaskConfig.SourceRoute + "playlists3" -> "Select * from music.playlists3", 30 | TaskConfig.SourceRoute + "playlists4" -> "Select * from music2.playlists4")).asJMap 31 | 32 | it should "validate and fail on invalid CassandraSource configuration" in { 33 | val cassandraSource = new CassandraSource() 34 | an[ConnectException] should be thrownBy { 35 | cassandraSource.start(Map.empty[String, String].asJava) 36 | } 37 | an[ConnectException] should be thrownBy { 38 | cassandraSource.start(Map("test" -> "").asJava) 39 | } 40 | an[ConnectException] should be thrownBy { 41 | cassandraSource.start(Map(TaskConfig.SourceRoute + "test" -> "").asJava) 42 | } 43 | an[ConnectException] should be thrownBy { 44 | cassandraSource.start(Map(TaskConfig.SourceRoute -> "query").asJava) 45 | } 46 | } 47 | 48 | it should "validate valid configuration of a CassandraSource on startup - one-to-one task strategy" in { 49 | val cassandraSource = new CassandraSource 50 | cassandraSource.start(config(TaskStrategy.OneToOne.key)) 51 | 52 | var taskConfigs = cassandraSource.taskConfigs(1) 53 | taskConfigs.size should be(1) 54 | taskConfigs.asScala forall { map => 55 | map.asScala.filterKeys(_.startsWith(TaskConfig.SourceRoute)).size == 4 && 56 | map.asScala.filterKeys(_.startsWith("cassandra.connection")).size == 3 && 57 | map.asScala.filterKeys(_.startsWith("cassandra.task.parallelism")).size == 1 58 | } should be (true) 59 | 60 | taskConfigs = cassandraSource.taskConfigs(2) 61 | taskConfigs.size should be(2) 62 | taskConfigs.asScala forall { map => 63 | map.asScala.filterKeys(_.startsWith(TaskConfig.SourceRoute)).size == 2 && 64 | map.asScala.filterKeys(_.startsWith("cassandra.connection")).size == 3 && 65 | map.asScala.filterKeys(_.startsWith("cassandra.task.parallelism")).size == 1 66 | } 67 | } 68 | 69 | it should "validate valid configuration of a CassandraSource on startup - one-to-many task strategy" in { 70 | val cassandraSource = new CassandraSource 71 | cassandraSource.start(config(TaskStrategy.OneToMany.key)) 72 | 73 | var taskConfigs = cassandraSource.taskConfigs(1) 74 | taskConfigs.size should be(1) 75 | taskConfigs.asScala forall { map => 76 | map.asScala.filterKeys(_.startsWith(TaskConfig.SourceRoute)).size == 4 && 77 | map.asScala.filterKeys(_.startsWith("cassandra.connection")).size == 3 && 78 | map.asScala.filterKeys(_.startsWith("cassandra.task.parallelism")).size == 1 79 | } 80 | 81 | taskConfigs = cassandraSource.taskConfigs(2) 82 | taskConfigs.size should be(2) 83 | taskConfigs.asScala forall { map => 84 | map.asScala.filterKeys(_.startsWith(TaskConfig.SourceRoute)).size == 2 && 85 | map.asScala.filterKeys(_.startsWith("cassandra.connection")).size == 3 && 86 | map.asScala.filterKeys(_.startsWith("cassandra.task.parallelism")).size == 1 87 | } 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /src/test/scala/com/tuplejump/kafka/connect/cassandra/CassandraSinkSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import scala.collection.JavaConverters._ 20 | import com.tuplejump.kafka.connect.cassandra.TaskConfig.TaskStrategy 21 | import org.apache.kafka.connect.errors.ConnectException 22 | 23 | class CassandraSinkSpec extends AbstractFlatSpec { 24 | 25 | private def config(taskStrategy: TaskStrategy) = (commonConfig ++ Map( 26 | TaskConfig.TaskParallelismStrategy -> taskStrategy.key, 27 | TaskConfig.SinkRoute + "devices.weather" -> "timeseries.hourly_weather1", 28 | TaskConfig.SinkRoute + "devices.user" -> "timeseries.user_devices1", 29 | TaskConfig.SinkRoute + "devices.weather2" -> "timeseries.hourly_weather2", 30 | TaskConfig.SinkRoute + "devices.user2" -> "timeseries.user_devices2")).asJMap 31 | 32 | it should "fail on invalid configurations" in { 33 | an[ConnectException] should be thrownBy { 34 | new CassandraSink().start(Map.empty[String, String].asJMap) 35 | } 36 | an[ConnectException] should be thrownBy { 37 | new CassandraSink().start(Map(TaskConfig.SinkRoute + "test" -> "").asJMap) 38 | } 39 | an[ConnectException] should be thrownBy { 40 | new CassandraSink().start(Map(TaskConfig.SinkRoute + "test" -> "test").asJMap) 41 | } 42 | an[ConnectException] should be thrownBy { 43 | new CassandraSink().start(Map(TaskConfig.SinkRoute + "test" -> ".test").asJMap) 44 | } 45 | an[ConnectException] should be thrownBy { 46 | new CassandraSink().start(Map("test" -> "ks.t").asJMap) 47 | } 48 | } 49 | 50 | it should "have taskConfigs with valid configurations with one to one" in { 51 | val cassandraSink: CassandraSink = new CassandraSink 52 | cassandraSink.start(config(TaskStrategy.OneToOne)) 53 | 54 | var taskConfigs = cassandraSink.taskConfigs(1) 55 | taskConfigs.size should be(1) 56 | taskConfigs.asScala forall { map => 57 | map.asScala.filterKeys(_.startsWith(TaskConfig.SinkRoute)).size == 4 && 58 | map.asScala.filterKeys(_.startsWith("cassandra.connection")).size == 3 && 59 | map.asScala.filterKeys(_.startsWith("cassandra.task.parallelism")).size == 1 60 | } should be(true) 61 | 62 | taskConfigs = cassandraSink.taskConfigs(2) 63 | taskConfigs.size should be(2) 64 | taskConfigs.asScala forall { map => 65 | map.asScala.filterKeys(_.startsWith(TaskConfig.SinkRoute)).size == 2 && 66 | map.asScala.filterKeys(_.startsWith("cassandra.connection")).size == 3 && 67 | map.asScala.filterKeys(_.startsWith("cassandra.task.parallelism")).size == 1 68 | } 69 | } 70 | 71 | it should "validate valid configuration of a CassandraSink on startup with one to many" in { 72 | val cassandraSink = new CassandraSink 73 | cassandraSink.start(config(TaskStrategy.OneToMany)) 74 | var taskConfigs = cassandraSink.taskConfigs(1) 75 | taskConfigs.size should be(1) 76 | taskConfigs.asScala forall { map => 77 | map.asScala.filterKeys(_.startsWith(TaskConfig.SinkRoute)).size == 4 && 78 | map.asScala.filterKeys(_.startsWith("cassandra.connection")).size == 3 && 79 | map.asScala.filterKeys(_.startsWith("cassandra.task.parallelism")).size == 1 80 | } should be(true) 81 | 82 | taskConfigs = cassandraSink.taskConfigs(2) 83 | taskConfigs.size should be(2) 84 | taskConfigs.asScala forall { map => 85 | map.asScala.filterKeys(_.startsWith(TaskConfig.SinkRoute)).size == 2 && 86 | map.asScala.filterKeys(_.startsWith("cassandra.connection")).size == 3 && 87 | map.asScala.filterKeys(_.startsWith("cassandra.task.parallelism")).size == 1 88 | } 89 | } 90 | } 91 | -------------------------------------------------------------------------------- /src/test/scala/com/tuplejump/kafka/connect/cassandra/SchemaSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import com.datastax.driver.core.{ DataType, TestUtil} 20 | import org.apache.kafka.connect.data.{Schema, SchemaBuilder, Struct} 21 | import org.apache.kafka.connect.sink.SinkRecord 22 | 23 | class SchemaSpec extends AbstractFlatSpec { 24 | 25 | it should "convert a struct schema with single field" in { 26 | val topic = "topicx" 27 | 28 | val sc = sinkConfig(topic, "keyspacex", "tablex", List("id")) 29 | sc.options.consistency should be (TaskConfig.DefaultSinkConsistency) 30 | sc.schema.columnNames should === (List("id")) 31 | sc.query.cql should be ("INSERT INTO keyspacex.tablex(id) VALUES(?)") 32 | 33 | val schema = SchemaBuilder.struct.name("record").version(1).field("id", Schema.INT32_SCHEMA).build 34 | val value = new Struct(schema).put("id", 1) 35 | val record = new SinkRecord(topic, 1, SchemaBuilder.struct.build, "key", schema, value, 0) 36 | 37 | sc.schema.route.topic should be (record.topic) 38 | sc.schema.route.keyspace should be ("keyspacex") 39 | sc.schema.route.table should be ("tablex") 40 | 41 | sc.schema is record should be (true) 42 | val query = record.as(sc.schema.namespace) 43 | query.cql should be("INSERT INTO keyspacex.tablex(id) VALUES(1)") 44 | } 45 | 46 | it should "convert a struct schema with multiple fields" in { 47 | val topic = "test_kfk" 48 | val sc = sinkConfig(topic, "keyspacex", "tablex", List("available", "name", "age")) 49 | 50 | val schema = SchemaBuilder.struct.name("record").version(1) 51 | .field("available", Schema.BOOLEAN_SCHEMA) 52 | .field("name", Schema.STRING_SCHEMA) 53 | .field("age", Schema.INT32_SCHEMA).build 54 | val value = new Struct(schema).put("name", "user").put("available", false).put("age", 15) 55 | val record = new SinkRecord("test_kfk", 1, SchemaBuilder.struct.build, "key", schema, value, 0) 56 | 57 | schema.asColumnNames should be (sc.schema.columnNames) 58 | 59 | sc.schema.route.topic should be (record.topic) 60 | sc.schema is record should be (true) 61 | 62 | sc.query.cql should be ("INSERT INTO keyspacex.tablex(available,name,age) VALUES(?,?,?)") 63 | val query = record.as(sc.schema.namespace) 64 | query.cql should be("INSERT INTO keyspacex.tablex(available,name,age) VALUES(false,'user',15)") 65 | } 66 | 67 | it should "convert cassandra column defs to a source schema" in { 68 | val colDef = Map( 69 | "id" -> DataType.cint(), 70 | "name" -> DataType.varchar()) 71 | 72 | val columns = TestUtil.getColumnDef(colDef) 73 | val expectedSchema = SchemaBuilder.struct() 74 | .field("id", Schema.INT32_SCHEMA) 75 | .field("name", Schema.STRING_SCHEMA).build() 76 | 77 | columns.asSchema should be(expectedSchema) 78 | } 79 | 80 | it should "convert kafka schema and struct to cassandra columns and schema mapping" in { 81 | import scala.collection.JavaConverters._ 82 | val topic = "a" 83 | val route = InternalConfig.Route(TaskConfig.SinkRoute + topic, "ks1.t1").get 84 | val schemaMap = new InternalConfig.Schema(route, Nil, Nil, Nil, List("available","name","age"), "") 85 | 86 | val schema = SchemaBuilder.struct.name("record").version(1) 87 | .field("available", Schema.BOOLEAN_SCHEMA) 88 | .field("name", Schema.STRING_SCHEMA) 89 | .field("age", Schema.INT32_SCHEMA).build 90 | val struct = new Struct(schema).put("name", "user").put("available", false).put("age", 15) 91 | val record = new SinkRecord(topic, 1, SchemaBuilder.struct.build, "key", schema, value, 0) 92 | 93 | schema.asColumnNames should ===(schemaMap.columnNames) 94 | schemaMap.columnNames should ===(schema.fields.asScala.map(_.name).toList) 95 | schemaMap is record should be (true) 96 | } 97 | } 98 | -------------------------------------------------------------------------------- /src/test/scala/com/tuplejump/kafka/connect/cassandra/TaskConfigSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import org.apache.kafka.common.config.ConfigException 20 | 21 | class TaskConfigSpec extends AbstractSpec { 22 | import TaskConfig._,InternalConfig._,Syntax._ 23 | 24 | "TaskConfig" must { 25 | "validate and create a new TaskConfig for a Sink with valid config" in { 26 | val cc = TaskConfig(sinkSchemas, sinkProperties(sinkTopicMap), sink) 27 | cc.sink.size should be(sinkSchemas.size) 28 | cc.source.isEmpty should be(true) 29 | } 30 | "validate and create a new Configuration for a Source with valid config" in { 31 | val query = "SELECT * FROM ks1.t1" 32 | val topic = topicList.head 33 | 34 | val schemas = for { 35 | route <- Route(TaskConfig.SourceRoute + topic, query) 36 | } yield Schema(route, Nil,Nil,Nil,Nil, query) 37 | 38 | val props = sourceProperties(query, topic) 39 | commonConfig ++ Map(TaskConfig.SourceRoute + topic -> query) 40 | val config = TaskConfig(schemas.toList, sourceProperties(query, topic), source) 41 | config.sink.isEmpty should be (true) 42 | config.source.size should be (1) 43 | for (ssc <- config.source) { 44 | ssc.schema.route.topic should === ("test1") 45 | ssc.schema.namespace should === ("ks1.t1") 46 | ssc.schema.route.keyspace should === ("ks1") 47 | ssc.schema.route.table should === ("t1") 48 | } 49 | } 50 | "validate and not create new Configuration source configs from invalid configuration" in { 51 | val schema = sinkSchemas.head 52 | 53 | an[ConfigException] should be thrownBy { 54 | TaskConfig(Nil, sourceProperties(schema.route.topic, "SELECT * FROM ks1.t1"), source) 55 | } 56 | } 57 | "validate and not create new Configuration sink configs from invalid configuration" in { 58 | val sinks = Map("a" -> "b.","a"->"b","a"->".b","a"->".","a"->"ab","a"->"") 59 | val props = sinkProperties(sinks) 60 | an[ConfigException] should be thrownBy { 61 | TaskConfig(Nil, props, sink) 62 | } 63 | } 64 | "validate and create new Topics for multiple valid entries" in { 65 | val config = TaskConfig(sinkSchemas, sinkProperties(sinkTopicMap), sink) 66 | config.sink.size should be (2) 67 | config.source.isEmpty should be (true) 68 | val consistency = DefaultSourceConsistency 69 | sinkSchemas.forall{ schema => 70 | val sc = SinkConfig(schema, PreparedQuery(schema), WriteOptions(DefaultSourceConsistency)) 71 | config.sink.contains(sc) 72 | } should be (true) 73 | } 74 | "validate and create new SinkConfigs from valid configurations" in { 75 | TaskConfig(sinkSchemas, sinkProperties(sinkTopicMap), sink) 76 | .sink.size should be (sinkSchemas.size) 77 | } 78 | "validate and not create sink config from invalid configurations" in { 79 | an[ConfigException] should be thrownBy { 80 | TaskConfig(Nil, sinkProperties(Map("a"->"b.","a"->"b","a"->".b","a"->".","a"->"ab","a"->"")), sink) 81 | } 82 | } 83 | "validate and not create source config with an empty or invalid query" in { 84 | an[ConfigException] should be thrownBy { 85 | TaskConfig(Nil, sourceProperties(query = "", "topic"), source) 86 | } 87 | an[ConfigException] should be thrownBy { 88 | TaskConfig(Nil, sourceProperties(query = "/", "topic"), source) 89 | } 90 | } 91 | "validate and not create source config with an empty or invalid topic" in { 92 | an[ConfigException] should be thrownBy { 93 | TaskConfig(Nil, sourceProperties(query = "SELECT * FROM test.playlists", ""), source) 94 | } 95 | } 96 | "not have configurations for source if set for just sink" in { 97 | an[ConfigException] should be thrownBy { 98 | TaskConfig(Nil, sourceProperties(query = "SELECT * FROM test.playlists", "topic"), source) 99 | } 100 | } 101 | "not have configurations for sink if set for just source" in { 102 | an[ConfigException] should be thrownBy { 103 | TaskConfig(Nil, sinkProperties(Map("a"->"k.t","a.b"->"k.t")), sink) 104 | } 105 | } 106 | "validate and create new Configuration for a source query" in { 107 | val s = sinkSchemas.head 108 | val query = "SELECT * FROM test.playlists" 109 | an[ConfigException] should be thrownBy { 110 | TaskConfig(List(s), sourceProperties(query, topicList.head), source) 111 | } 112 | } 113 | } 114 | } 115 | -------------------------------------------------------------------------------- /scalastyle-config.xml: -------------------------------------------------------------------------------- 1 | 2 | Scalastyle configuration 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | false 97 | 98 | 99 | 100 | -------------------------------------------------------------------------------- /src/it/scala/com/tuplejump/kafka/connect/cassandra/CassandraSourceTaskSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to Tuplejump Software Pvt. Ltd. under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. Tuplejump Software Pvt. Ltd. licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | * 18 | */ 19 | 20 | package com.tuplejump.kafka.connect.cassandra 21 | 22 | import scala.concurrent.duration._ 23 | import scala.collection.JavaConverters._ 24 | import org.apache.kafka.connect.source.SourceTaskContext 25 | 26 | class CassandraSourceTaskSpec extends AbstractFlatSpec { 27 | 28 | val query = "SELECT * FROM test.playlists" 29 | val topic = "test" 30 | val config = sourceProperties(query, topic) 31 | 32 | def insertStmt(time: Long): String = { 33 | "INSERT INTO test.event_store(app_id,event_type,subscription_type,event_ts) " + 34 | s"VALUES ('website','renewal','annual',$time)" 35 | } 36 | 37 | it should "start source task" in { 38 | val sourceTask = new CassandraSourceTask() 39 | val mockContext = mock[SourceTaskContext] 40 | 41 | sourceTask.initialize(mockContext) 42 | 43 | sourceTask.start(config.asJava) 44 | Option(sourceTask.session).isDefined should be(true) 45 | sourceTask.stop() 46 | } 47 | 48 | it should "fetch records from cassandra in bulk" in { 49 | val sourceTask = new CassandraSourceTask() 50 | val mockContext = mock[SourceTaskContext] 51 | 52 | sourceTask.initialize(mockContext) 53 | sourceTask.start(config.asJava) 54 | 55 | val result = sourceTask.poll() 56 | 57 | result.size() should be(4) 58 | 59 | sourceTask.stop() 60 | } 61 | 62 | 63 | it should "fetch only new records from cassandra" in { 64 | val timeFunctionQuery = 65 | "SELECT * FROM test.event_store " + 66 | "WHERE app_id='website' AND event_type='renewal' " + 67 | "AND event_ts >= previousTime()" 68 | 69 | val config = sourceProperties(timeFunctionQuery, "events") 70 | 71 | val sourceTask = new CassandraSourceTask() 72 | val mockContext = mock[SourceTaskContext] 73 | 74 | sourceTask.initialize(mockContext) 75 | sourceTask.start(config.asJava) 76 | 77 | val oneHrAgo = System.nanoTime + HOURS.toMillis(1) 78 | sourceTask.session.execute(insertStmt(oneHrAgo)) 79 | 80 | // was sourceTask.poll().size() should be(0) 81 | val t1 = sourceTask.poll() 82 | //check size and uniqueness, currently not unique and size 1, not 0 83 | 84 | val oneHrLater = System.nanoTime + HOURS.toMillis(1) 85 | sourceTask.session.execute(insertStmt(oneHrLater)) 86 | 87 | // was sourceTask.poll().size() should be(1) 88 | val t2 = sourceTask.poll 89 | //check size and uniqueness, currently not unique and size 1, not 0 90 | 91 | sourceTask.stop() 92 | } 93 | 94 | it should "fetch records from cassandra in given pollInterval" in { 95 | val timeFunctionQuery = 96 | """SELECT * FROM test.event_store WHERE app_id='website' AND event_type='renewal' 97 | | AND event_ts >= previousTime() AND event_ts <= currentTime()""".stripMargin 98 | 99 | val config = sourceProperties(timeFunctionQuery, "events") 100 | 101 | val sourceTask = new CassandraSourceTask() 102 | val mockContext = mock[SourceTaskContext] 103 | 104 | sourceTask.initialize(mockContext) 105 | sourceTask.start(config.asJava) 106 | 107 | val oneHrLater = System.currentTimeMillis() + HOURS.toMillis(1) 108 | sourceTask.session.execute(insertStmt(oneHrLater)) 109 | 110 | val fewSecLater = System.currentTimeMillis() + SECONDS.toMillis(2) 111 | sourceTask.session.execute(insertStmt(fewSecLater)) 112 | 113 | val result = sourceTask.poll() 114 | 115 | result.size() should be(1) 116 | 117 | sourceTask.stop() 118 | } 119 | 120 | it should "fetch records from cassandra" in { 121 | val query = "SELECT * FROM test.playlists" 122 | val topic = "test" 123 | val config = sourceProperties(query, topic) 124 | 125 | val task = new CassandraSourceTask() 126 | val mockContext = mock[SourceTaskContext] 127 | 128 | task.initialize(mockContext) 129 | task.start(config.asJava) 130 | 131 | val existing = task.poll() 132 | existing.size() should be(4) 133 | existing.asScala.forall(_.topic == topic) should be(true) 134 | 135 | val updates = Seq( 136 | """INSERT INTO test.playlists (id, song_order, song_id, title, artist, album) 137 | VALUES (5, 5, 65482, 'The Lemon Song', 'Led Zeppelin', 'Led Zeppelin II')""", 138 | """INSERT INTO test.playlists (id, song_order, song_id, title, artist, album) 139 | VALUES (6, 6, 45015, 'Monkey Man', 'Rolling Stones', 'Let It Bleed')""" 140 | ) 141 | 142 | updates foreach (task.session.execute) 143 | val records = task.poll.asScala 144 | records.forall(_.topic == topic) should be(true) 145 | //is existing.size + updates.size until: 146 | //TODO: https://tuplejump.atlassian.net/browse/DB-56 timeuuid,timestamp 147 | //records.size should be(updates.size) 148 | //needs to be true: task.poll.size should be (0) 149 | 150 | task.stop() 151 | } 152 | } 153 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Kafka Connect Cassandra [![GitHub license](https://img.shields.io/badge/license-Apache%20V2-green.svg)](https://github.com/tuplejump/kafka-connect-cassandra/blob/master/LICENSE) [![Build Status](https://travis-ci.org/tuplejump/kafka-connect-cassandra.svg?branch=master)](https://travis-ci.org/tuplejump/kafka-connect-cassandra?branch=master) 2 | Kafka Connect Cassandra Connector. This project includes source & sink connectors. 3 | 4 | ## Release Status 5 | Experimental phase. 6 | 7 | ## Table of Contents 8 | 9 | * [Usage](#usage) 10 | * [Running Examples](#running-example) 11 | * [CassandraSource](#cassandrasource) 12 | * [Supported CQL Types](#cql-types-supported) 13 | * [CassandraSink](#cassandrasink) 14 | * [Configuration](#configuration) 15 | * [Connect Properties](#connect-properties-for-both-source-and-sink) 16 | * [Source Connect Properties](#source-connect-properties) 17 | * [Sink Connect Properties](#sink-connect-properties) 18 | * [Building From Source](#building-from-source) 19 | 20 | ## Usage 21 | The project can be used in two ways: 22 | 23 | 1. as a library in another project, 24 | 25 | "kafka-connect-cassandra" is published on maven central by Tuplejump. It can be defined as a dependency in the build file. 26 | For example, with SBT, 27 | 28 | ``` 29 | libraryDependencies += "com.tuplejump" %% "kafka-connect-cassandra" % "0.0.7" 30 | ``` 31 | 32 | 2. using jar with Kafka Connect 33 | 34 | Download the [jar](http://downloads.tuplejump.com/kafka-connect-cassandra-assembly-0.0.7.jar) and copy it to 'KAFKA_HOME/libs' 35 | 36 | ## Running Examples 37 | The example uses CassandraSource and loads data from `demo.event_store`. This data is then saved in another table `demo.event_store_sink` using CassandraSink. 38 | To run the example, execute `examples/start.sh` in the project directory. 39 | 40 | This script starts Cassandra, Zookeper, Kafka Server and then initiates a Kafka Connect command using the Source and Sink Configuration specified in `examples/config`. 41 | Once the script execution is completed, the data from `demo.event_store` will be available in `demo.event_store_sink`. You can insert a few rows in `demo.event_store` to see this. 42 | 43 | To stop the processes started for running the example, execute `examples/stop.sh` 44 | 45 | Note: On running the example repeatedly, an error is thrown that topic `demo` already exists but it won't block the execution 46 | 47 | ## CassandraSource 48 | It polls Cassandra with specified query. Using this, data can be fetched from Cassandra in two modes: 49 | 50 | 1. bulk 51 | 2. timestamp based 52 | 53 | The modes change automatically based on the query. For example, 54 | 55 | ```sql 56 | SELECT * FROM userlog ; //bulk 57 | 58 | SELECT * FROM userlog WHERE ts > previousTime() ; //timestamp based 59 | 60 | SELECT * FROM userlog WHERE ts = currentTime() ; //timestamp based 61 | 62 | SELECT * FROM userlog WHERE ts >= previousTime() AND ts <= currentTime() ; //timestamp based 63 | ``` 64 | 65 | Here, `previousTime()` and `currentTime()` are replaced prior to fetching the data. 66 | 67 | ### CQL Types Supported 68 | 69 | | CQL Type | Schema Type | 70 | |----------|------------| 71 | | ASCII | STRING | 72 | | VARCHAR | STRING | 73 | | TEXT | STRING | 74 | | BIGINT | INT64 | 75 | | COUNTER | INT64 | 76 | | BOOLEAN | BOOLEAN | 77 | | DECIMAL | FLOAT64 | 78 | | DOUBLE | FLOAT64 | 79 | | FLOAT | FLOAT32 | 80 | | TIMESTAMP | TIMESTAMP | 81 | | VARINT | INT64 | 82 | 83 | All the others (BLOB, INET, UUID, TIMEUUID, LIST, SET, MAP, CUSTOM, UDT, TUPLE, SMALLINT, TINYINT, DATE,TIME) are currently NOT supported. 84 | 85 | ## CassandraSink 86 | It stores Kafka SinkRecord in Cassandra tables. Currently, we only support STRUCT type in the SinkRecord. 87 | The STRUCT can have multiple fields with primitive fieldtypes. We assume one-to-one mapping between the column names in the Cassandra sink table and the field names. 88 | 89 | Say, the SinkRecords has the following STRUCT value 90 | ``` 91 | { 92 | 'id': 1, 93 | 'username': 'user1', 94 | 'text': 'This is my first tweet' 95 | } 96 | ``` 97 | 98 | Then the Cassandra table should have the columns - id, username, text 99 | 100 | Note: The library does not create the Cassandra tables - users are expected to create those before starting the sink 101 | 102 | ## Configuration 103 | 104 | Refer `examples/config` for sample configuration files 105 | 106 | 107 | ### Connect Properties (for both Source and Sink) 108 | | name | description | default value | 109 | |-------- |----------------------------|-----------------------| 110 | | name | Unique name for the connector. Attempting to register again with the same name will fail. | | 111 | | connector.class | The Java class for the connector, `com.tuplejump.kafka.connect.cassandra.CassandraSource` or `com.tuplejump.kafka.connect.cassandra.CassandraSink` | | 112 | | tasks.max | The maximum number of tasks that should be created for this connector. The connector may create fewer tasks if it cannot achieve this level of parallelism. | | 113 | | | **Cassandra Properties (for both Source and Sink)** | | 114 | | cassandra.connection.host | The host name or IP address to which the Cassandra native transport is bound. | localhost | 115 | | cassandra.connection.port | Port for native client protocol connections. | 9042 | 116 | | cassandra.connection.auth.username | Cassandra username | '' | 117 | | cassandra.connection.auth.password | Cassandra password | '' | 118 | | cassandra.connection.timeout.ms | Connection timeout duration (in ms) | 8000 | 119 | | cassandra.connection.read.timeout | Read timeout duration (in ms) | 120000 | 120 | | cassandra.connection.reconnect.delay.min | Minimum period of time (in ms) to wait before reconnecting to a dead node. | 1000 | 121 | | cassandra.connection.reconnect.delay.max | Maximum period of time (in ms) to wait before reconnecting to a dead node. | 60000 | 122 | | cassandra.connection.consistency | Consistency level. Values allowed can be seen [here](http://docs.datastax.com/en/drivers/java/2.1/com/datastax/driver/core/ConsistencyLevel.html) | QUORUM | 123 | | cassandra.task.parallelism | Can be one of the values: one-to-one (One topic-to-keyspace.table per task) or one-to-many (N-topic-to-keyspace.table per task) | one-to-one | 124 | ### Source Connect Properties 125 | | name | description | default value | 126 | |-------- |----------------------------|-----------------------| 127 | | cassandra.source.route.\ | The Select Query to get the data. (Refer CassandraSource documentation for more details) | | 128 | | cassandra.source.poll.interval | Frequency in ms to poll for new data in each table. | 60000 | 129 | | cassandra.source.fetch.size | Number of CQL rows to fetch in a single round-trip to Cassandra. | 1000 | 130 | ### Sink Connect Properties 131 | | name | description | default value | 132 | |-------- |----------------------------|-----------------------| 133 | | cassandra.sink.route.\ | The table to write the SinkRecords to, \.\ | | 134 | | cassandra.sink.consistency | The consistency level for writes to Cassandra. | LOCAL_QUORUM | 135 | 136 | 137 | ## Building from Source 138 | The project requires SBT to build from source. Execute the following command in the project directory, 139 | 140 | ``` 141 | sbt package 142 | sbt assembly // to generate the jar with all the dependencies 143 | ``` 144 | This will build against Scala 2.11.7 by default. You can override this with: 145 | 146 | ``` 147 | sbt -Dscala.version=2.10.6 assembly 148 | ``` 149 | 150 | Or to build against multiple Scala versions, 151 | ``` 152 | sbt +package 153 | ``` 154 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/ConnectorLike.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import java.util.{List => JList, Map => JMap} 20 | import java.util.concurrent.atomic.AtomicBoolean 21 | 22 | import scala.collection.immutable 23 | import scala.util.control.NonFatal 24 | import org.apache.kafka.connect.util.ConnectorUtils 25 | import org.apache.kafka.connect.connector.{Connector, Task} 26 | import org.apache.kafka.common.config.ConfigException 27 | import org.apache.kafka.connect.errors.ConnectException 28 | import com.datastax.driver.core.Session 29 | 30 | /** All Cassandra source, sink and task instances delegate to this 31 | * trait for configuration. 32 | * 33 | * INTERNAL API. 34 | */ 35 | private[kafka] trait ConnectorLike extends Logging { 36 | import InternalConfig.Route 37 | import TaskConfig.TaskStrategy 38 | 39 | protected final val version: String = CassandraConnectorInfo.version 40 | 41 | /* filtered for cassandra.source.* or cassandra.sink.*. */ 42 | protected var configT: Map[String,String] = Map.empty 43 | 44 | /** Initialized as the default, user can override. */ 45 | protected var taskStrategy: TaskStrategy = TaskStrategy.OneToOne 46 | 47 | protected var routes: List[Route] = Nil 48 | 49 | /** Reads in the user provided configuration for a given source or sink. 50 | * Fails fast on [[org.apache.kafka.connect.connector.Connector]] `start` if not valid. 51 | * 52 | * Because user params can be a mine field on bugs farther down stream we attempt 53 | * to catch as many up front as we can. 54 | * 55 | * @param conf the user config 56 | * @throws org.apache.kafka.common.config.ConfigException if requirements not met 57 | */ 58 | protected def configure(conf: immutable.Map[String, String], taskClass: Class[_ <: Task]): Unit = { 59 | val valid = conf.filterNonEmpty 60 | logger.debug(s"Configuration validation starting with ${valid.size} from ${conf.size}") 61 | 62 | val key = taskClass match { 63 | case `source` => TaskConfig.SourceNamespace 64 | case `sink` => TaskConfig.SinkNamespace 65 | case _ => "" 66 | } 67 | 68 | /* Filters by cassandra.source.* or cassandra.sink.* */ 69 | configT = valid.common ++ valid.filterKeys(_.startsWith(key)) 70 | logger.debug(s"Configuring [${configT.size}] from $key.*") 71 | 72 | try { 73 | routes = configT.flatMap { case (k,v) => Route(k, v)}.toList 74 | configRequire(routes.nonEmpty, "At least one topic to keyspace.table should be configured.") 75 | 76 | val configured = configT.count(_._1.contains(".route.")) 77 | configRequire(routes.size == configured, 78 | s"Expected $configured routes based on config, ${routes.size} were valid: ${routes.mkString(",")}") 79 | } 80 | catch { case e: ConfigException => throw new ConnectException( 81 | s"Unable to start due to configuration error, ${e.getMessage}", e) 82 | } 83 | 84 | taskStrategy = TaskStrategy(conf.common) 85 | 86 | logger.info(s"Configured ${routes.size} Kafka - Cassandra mappings.") 87 | } 88 | } 89 | 90 | /** A CassandraConnector is either a [[CassandraSource]] or a [[CassandraSink]]. 91 | * INTERNAL API. 92 | */ 93 | private[kafka] trait CassandraConnector extends Connector with ConnectorLike { 94 | 95 | /* The original values passed in by a user. */ 96 | protected var originals: JMap[String, String] = Map.empty[String,String].asJMap 97 | 98 | override def start(props: JMap[String, String]): Unit = { 99 | connectRequire(!props.isEmpty, "Configurations must not be empty") 100 | originals = props 101 | configure(props.immutable, taskClass) 102 | } 103 | 104 | /** Task parallelism strategy configurable by user. Determines the number of input tasks, 105 | * and then divide them up. 106 | */ 107 | override def taskConfigs(maxTasks: Int): JList[JMap[String, String]] = { 108 | import scala.collection.JavaConverters._ 109 | import TaskConfig.TaskStrategy._ 110 | 111 | val (routeConfs, common) = configT partition(_._1.contains(".route.")) 112 | 113 | taskStrategy match { 114 | case OneToOne if routeConfs.size < maxTasks => 115 | logger.debug(s"Creating $maxTasks task configs with task parallelism strategy 'OneToOne'.") 116 | (for ((k,v) <- routeConfs.toList) yield (common ++ Map(k -> v)).asJMap).asJava 117 | 118 | case _ => 119 | logger.debug(s"Creating $maxTasks task configs with task parallelism strategy 'OneToMany'.") 120 | val numGroups = Math.min(routeConfs.size, maxTasks) 121 | val tablesGrouped = ConnectorUtils.groupPartitions(routeConfs.toList.asJava, numGroups) 122 | (for (group <- tablesGrouped.asScala) yield 123 | (common ++ (for ((k,v) <- group.asScala) yield k -> v)).asJMap).asJava 124 | } 125 | } 126 | 127 | override def stop(): Unit = { 128 | logger.info(s"${getClass.getSimpleName} shutting down.") 129 | } 130 | } 131 | 132 | /** INTERNAL API. */ 133 | private[kafka] trait CassandraTask extends Task with ConnectorLike with CassandraClusterApi { 134 | 135 | private val runnable = new AtomicBoolean(false) 136 | 137 | private var _session: Option[Session] = None 138 | 139 | /** The parsed values based on the config map passed in on start. 140 | * The Kafka mutable java map is converted to a scala immutable, parsed based on 141 | * whether this is for a source or sink or task thereof, then validated by type. 142 | * 143 | * If a source task, this will have one or more SourceConfigs. 144 | * If a sink task, this will have one or more SinkConfigs. 145 | * Currently we hand each task a unique topic to keyspace and table mapping. 146 | * Roadmap: Parallism partitioning strategy configurable by user. 147 | */ 148 | protected var taskConfig = TaskConfig.Empty 149 | 150 | protected var cluster: Option[CassandraCluster] = None 151 | 152 | protected def taskClass: Class[_ <: Task] 153 | 154 | def session: Session = _session.getOrElse(throw new ConnectException( 155 | getClass.getSimpleName + " has not been started yet or is not properly configured.")) 156 | 157 | /** Restarts call stop() then start(). 158 | * 159 | * @param conf the filtered config from the Connector 160 | */ 161 | override def start(conf: JMap[String, String]): Unit = { 162 | val config = conf.immutable 163 | configure(config, taskClass) 164 | 165 | val connector = CassandraCluster(config) 166 | 167 | try { 168 | _session = Some(connector.session) 169 | cluster = Some(connector) 170 | 171 | val routeMap = routes.map(r => r -> tableFor(r)).toMap 172 | taskConfig = TaskConfig(routeMap, configT, taskClass) 173 | 174 | runnable set true 175 | status() 176 | } catch { case NonFatal(e) => 177 | logger.error(s"Unable to start ${getClass.getSimpleName}, shutting down.", e) 178 | } 179 | } 180 | 181 | override def stop(): Unit = { 182 | runnable set false 183 | status() 184 | cluster foreach (_.shutdown()) 185 | } 186 | 187 | private def status(): Unit = { 188 | val status = if (runnable.get) s"starting with ${routes.size} routes" else s"shutting down" 189 | logger.info(s"${getClass.getSimpleName} $status.")//task id? 190 | } 191 | } 192 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect 18 | 19 | /** Common package operations. */ 20 | package object cassandra { 21 | import java.util.{List => JList, Map => JMap, Date => JDate} 22 | 23 | import scala.collection.JavaConverters._ 24 | import com.datastax.driver.core._ 25 | import com.datastax.driver.core.DataType.{Name => CQLType} 26 | import com.datastax.driver.core.Row 27 | import org.apache.kafka.connect.data.{Timestamp, Struct, SchemaBuilder, Schema} 28 | import org.apache.kafka.connect.errors.{ConnectException, DataException} 29 | import org.apache.kafka.connect.source.SourceRecord 30 | import org.apache.kafka.connect.data.Schema.Type._ 31 | import org.apache.kafka.connect.sink.SinkRecord 32 | import org.apache.kafka.common.config.ConfigException 33 | import Types._,Syntax._ 34 | 35 | lazy val EmptySources: JList[SourceRecord] = List.empty[SourceRecord].asJava 36 | 37 | lazy val EmptyJMap: JMap[String, Any] = Map.empty[String, Any].asJava 38 | 39 | lazy val EmptyJTaskConfigs: JList[JMap[String, String]] = List(Map.empty[String, String].asJava).asJava 40 | 41 | val source = classOf[CassandraSourceTask] 42 | 43 | val sink = classOf[CassandraSinkTask] 44 | 45 | implicit class ColumnDefinitionsOps(columns: ColumnDefinitions) { 46 | 47 | def asSchema: Schema = { 48 | val builder = SchemaBuilder.struct 49 | for (column <- columns.asList.asScala) builder.field(column.getName, fieldType(column)) 50 | builder.build() 51 | } 52 | 53 | private def fieldType(column: ColumnDefinitions.Definition): Schema = 54 | column.getType.getName match { 55 | case CQLType.ASCII | CQLType.VARCHAR | CQLType.TEXT => Schema.STRING_SCHEMA 56 | case CQLType.BIGINT | CQLType.COUNTER => Schema.INT64_SCHEMA 57 | case CQLType.BOOLEAN => Schema.BOOLEAN_SCHEMA 58 | case CQLType.DECIMAL | CQLType.DOUBLE => Schema.FLOAT64_SCHEMA 59 | case CQLType.FLOAT => Schema.FLOAT32_SCHEMA 60 | case CQLType.INT => Schema.INT32_SCHEMA 61 | case CQLType.TIMESTAMP => Timestamp.SCHEMA 62 | case CQLType.VARINT => Schema.INT64_SCHEMA 63 | case other => 64 | //TODO 65 | //BLOB,INET,UUID,TIMEUUID,LIST,SET,MAP,CUSTOM,UDT,TUPLE,SMALLINT,TINYINT,DATE,TIME 66 | throw new DataException(s"Querying for type $other is not supported") 67 | } 68 | } 69 | 70 | implicit class SinkRecordOps(record: SinkRecord) { 71 | 72 | def as(namespace: String): SinkQuery = { 73 | val schema = record.valueSchema 74 | val columnNames = schema.asColumnNames 75 | val columnValues = schema.`type`() match { 76 | case STRUCT => 77 | val struct: Struct = record.value.asInstanceOf[Struct] 78 | columnNames.map(convert(schema, struct, _)).mkString(",") 79 | case other => throw new DataException( 80 | s"Unable to create insert statement with unsupported value schema type $other.") 81 | } 82 | SinkQuery(namespace, columnNames, columnValues) 83 | } 84 | 85 | /* TODO support all types. */ 86 | def convert(schema: Schema, result: Struct, col: String): AnyRef = 87 | schema.field(col).schema match { 88 | case x if x.`type`() == Schema.STRING_SCHEMA.`type`() => 89 | s"'${result.get(col).toString}'" 90 | case x if x.name() == Timestamp.LOGICAL_NAME => 91 | val time = Timestamp.fromLogical(x, result.get(col).asInstanceOf[JDate]) 92 | s"$time" 93 | case y => 94 | result.get(col) 95 | } 96 | 97 | def asColumnNames: List[ColumnName] = 98 | record.valueSchema.asColumnNames 99 | 100 | } 101 | 102 | implicit class RowOps(row: Row) { 103 | 104 | def as(topic: TopicName, partition: JMap[String, Any], offset: JMap[String, Any]): SourceRecord = { 105 | val schema = row.asSchema 106 | val struct = schema.asStruct(row) 107 | new SourceRecord(partition, offset, topic, schema, struct) 108 | } 109 | 110 | def asSchema: Schema = row.getColumnDefinitions.asSchema 111 | } 112 | 113 | implicit class SchemaOps(schema: Schema) { 114 | 115 | def asColumnNames: List[String] = 116 | schema.fields.asScala.map(_.name).toList 117 | 118 | def asStruct(row: Row): Struct = { 119 | val struct: Struct = new Struct(schema) 120 | 121 | for (field <- schema.fields.asScala) { 122 | val colName: String = field.name 123 | struct.put(colName, row.getObject(colName)) 124 | } 125 | 126 | struct 127 | } 128 | } 129 | 130 | implicit class TableMetadataOps(table: TableMetadata) { 131 | import Types._,Syntax._ 132 | 133 | def cql: Cql = table.asCQLQuery.toLowerCase 134 | 135 | def hasOrderClause(orderClause: OrderClause): Boolean = 136 | cql.contains(orderClause.toLowerCase) 137 | 138 | def clusteringColumns: List[ColumnMetadata] = 139 | table.getClusteringColumns.asScala.toList 140 | 141 | def hasClusteringOrder: Boolean = 142 | clusteringColumns.nonEmpty && hasOrderClause(ClusteringOrderBy) 143 | 144 | def partitionKeyNames: List[ColumnName] = 145 | table.getPartitionKey.asScala.toList.map(_.getName) 146 | 147 | def primaryKeyNames: List[ColumnName] = 148 | table.getPrimaryKey.asScala.toList.map(_.getName) 149 | 150 | def clusteringColumnNames: List[ColumnName] = 151 | clusteringColumns.map(_.getName) 152 | 153 | def columnNames: List[ColumnName] = 154 | table.getColumns.asScala.toList.map(_.getName) 155 | } 156 | 157 | implicit class ConfigOps(config: Map[String, String]) { 158 | def asJMap: JMap[String, String] = config.asJava 159 | 160 | /** Returns the value or `default` [[A]] from `config`.*/ 161 | def valueOr[A](key: Key, func: (String) => A, default: A): A = 162 | valueNonEmpty(key) map(v => func(v.trim)) getOrElse default 163 | 164 | def valueOrThrow(key: Key, message: String): Option[String] = 165 | valueNonEmpty(key).orElse(throw new ConfigException(message)) 166 | 167 | /** Returns the value from `config` or None if not exists. */ 168 | def valueNonEmpty(key: String): Option[String] = 169 | config.get(key) collect { case value if value.trim.nonEmpty => value.trim } 170 | 171 | /** Returns a new map of only the non empty key-value pairs. */ 172 | def filterNonEmpty: Map[String, String] = 173 | config.filter { case (k,v) => k.nonEmpty && v.nonEmpty } 174 | 175 | def common: Map[String,String] = { 176 | config.filterKeys(_.startsWith("cassandra.connection")) ++ 177 | config.filterKeys(_.startsWith("cassandra.task")) 178 | } 179 | } 180 | 181 | implicit class JavaOps(config: JMap[String, String]) { 182 | /** Kafka Connect API provides stringly-typed params as a mutable map. */ 183 | def immutable: Map[String, String] = 184 | Map.empty[String, String] ++ config.asScala 185 | } 186 | 187 | implicit final def configRequire(requirement: Boolean, message: => Any) { 188 | if (!requirement) throw new ConfigException(s"Requirement failed: $message") 189 | } 190 | implicit final def connectRequire(requirement: Boolean, message: => Any) { 191 | if (!requirement) throw new ConnectException(s"Requirement failed: $message") 192 | } 193 | 194 | } 195 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/CassandraCluster.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import java.net.{ConnectException, InetAddress} 20 | 21 | import scala.util.Try 22 | import scala.util.control.NonFatal 23 | import com.datastax.driver.core._ 24 | import com.datastax.driver.core.Session 25 | import com.datastax.driver.core.policies.ExponentialReconnectionPolicy 26 | 27 | /** TODO refactor to control the # of cluster objs, sessions, cache sessions, etc. 28 | * 29 | * @param seedNodes Addresses of hosts that are deemed contact points. 30 | * Cassandra nodes use this list of hosts to find each 31 | * other and learn the topology of the ring. 32 | * @param port the port to use to connect to Cassanrda seed nodes 33 | * @param compression LZ4, Snappy, and Deflate 34 | */ 35 | private[cassandra] 36 | final class CassandraCluster(val seedNodes: Seq[InetAddress], 37 | val port: Int, 38 | val authProvider: AuthProvider, 39 | val connectionTimeout: Int, 40 | val readTimeout: Int, 41 | val minReconnectDelay: Int, 42 | val maxReconnectDelay: Int, 43 | val compression: ProtocolOptions.Compression, 44 | val defaultConsistencyLevel: ConsistencyLevel 45 | ) extends CassandraClusterApi with Logging { 46 | 47 | sys.runtime.addShutdownHook(new Thread(s"Shutting down any open cassandra sessions.") { 48 | override def run(): Unit = { 49 | logger.info("Shutdown starting.") 50 | shutdown() 51 | } 52 | }) 53 | 54 | private var _session: Option[Session] = None 55 | 56 | private[this] lazy val socketOptions = new SocketOptions() 57 | .setConnectTimeoutMillis(connectionTimeout) 58 | .setReadTimeoutMillis(readTimeout) 59 | 60 | private[this] lazy val nettyOptions = 61 | new NettyOptions() //TODO 62 | 63 | private[this] lazy val queryOptions = 64 | new QueryOptions().setConsistencyLevel(defaultConsistencyLevel) 65 | 66 | lazy val session: Session = _session getOrElse { 67 | /* .withLoadBalancingPolicy().withSSL() */ 68 | val cluster = Cluster.builder 69 | .addContactPoints(seedNodes: _*) 70 | .withPort(port) 71 | .withAuthProvider(authProvider) 72 | .withReconnectionPolicy(new ExponentialReconnectionPolicy(minReconnectDelay, maxReconnectDelay)) 73 | .withCompression(compression) 74 | .withSocketOptions(socketOptions) 75 | .withQueryOptions(queryOptions) 76 | .withNettyOptions(nettyOptions).build 77 | 78 | try { 79 | 80 | val clusterName = cluster.getMetadata.getClusterName 81 | logger.info(s"Connected to Cassandra cluster: $clusterName") 82 | 83 | val s = cluster.connect() 84 | logger.info(s"$s created.") 85 | _session = Some(s) 86 | s 87 | } catch { 88 | case NonFatal(e) => 89 | cluster.close 90 | throw new ConnectException( 91 | s"Unable to create session with hosts[$seedNodes], port[$port]: ${e.getMessage}") 92 | } 93 | } 94 | 95 | def clusterVersions: Set[VersionNumber] = { 96 | import scala.collection.JavaConverters._ 97 | 98 | session.getCluster.getMetadata.getAllHosts 99 | .asScala.map(_.getCassandraVersion).toSet 100 | } 101 | 102 | //can be better & cleaner 103 | private[cassandra] def shutdown(): Unit = _session foreach { s => 104 | if (!s.isClosed || !s.getCluster.isClosed) { 105 | val cluster = s.getCluster 106 | try { 107 | val clusterName = cluster.getMetadata.getClusterName 108 | s.close() 109 | cluster.close() 110 | logger.info(s"Disconnected from Cassandra cluster: $clusterName") 111 | } catch { 112 | case NonFatal(e) => logger.error("Error during shutdown.", e) 113 | } finally session.getCluster.close() 114 | } 115 | } 116 | } 117 | 118 | object CassandraCluster extends ConnectionValidation { 119 | 120 | /** Creates a new CassandraCluster instance with the default configuration for local use. */ 121 | def local: CassandraCluster = apply(Map.empty[String, String]) 122 | 123 | /** Creates a new CassandraCluster instance with the user configuration. 124 | * Default config is used if not provided. 125 | * 126 | * TODO config validation 127 | * 128 | * @param config the user provided configuration 129 | */ 130 | def apply(config: Map[String, String]): CassandraCluster = { 131 | import InternalConfig._ 132 | 133 | new CassandraCluster( 134 | seedNodes = hosts(config.getOrElse(ConnectionHosts, DefaultHosts)), 135 | port = config.valueOr[Int](ConnectionPort, toInt, DefaultPort), 136 | authProvider = (for { 137 | user <- config.valueNonEmpty(ConnectionUsername) 138 | pass <- config.valueNonEmpty(ConnectionPassword) 139 | } yield new PlainTextAuthProvider(user, pass)).getOrElse(AuthProvider.NONE), 140 | connectionTimeout = config.valueOr[Int](ConnectionTimeout, toInt, DefaultConnectionTimeout), 141 | readTimeout = config.valueOr[Int](ConnectionReadTimeout, toInt, DefaultReadTimeout), 142 | minReconnectDelay = config.valueOr[Int](ConnectionMinReconnectDelay, toInt, DefaultMinReconnectDelay), 143 | maxReconnectDelay = config.valueOr[Int](ConnectionMaxReconnectDelay, toInt, DefaultMaxReconnectDelay), 144 | compression = compressionV(config), 145 | defaultConsistencyLevel = config.valueOr[ConsistencyLevel]( 146 | ConnectionConsistency, toConsistency, DefaultConsistency) 147 | ) 148 | } 149 | 150 | /* Config to read from the user's config (Roadmap: or deploy environment or -D java system properties) */ 151 | 152 | /** Cassandra hosts: contact points to connect to the Cassandra cluster. 153 | * A comma separated list of seed nodes may also be used: "127.0.0.1,192.168.0.1". */ 154 | val ConnectionHosts = "cassandra.connection.host" 155 | val DefaultHosts = "localhost" 156 | 157 | /** Cassandra native connection port. */ 158 | val ConnectionPort = "cassandra.connection.port" 159 | val DefaultPort = ProtocolOptions.DEFAULT_PORT 160 | 161 | /** Auth */ 162 | val ConnectionUsername = "cassandra.connection.auth.username" 163 | val ConnectionPassword = "cassandra.connection.auth.password" 164 | 165 | /** Maximum period of time to attempt connecting to a node. */ 166 | val ConnectionTimeout = "cassandra.connection.timeout.ms" 167 | val DefaultConnectionTimeout = 8000 168 | 169 | /** Maximum period of time to wait for a read to return. */ 170 | val ConnectionReadTimeout = "cassandra.connection.read.timeout" 171 | val DefaultReadTimeout = 120000 172 | 173 | /** Period of time, in ms, to keep unused connections open. */ 174 | val ConnectionKeepAliveMillis = "cassandra.connection.keep_alive" //TODO 175 | 176 | /** Minimum period of time to wait before reconnecting to a dead node. */ 177 | val ConnectionMinReconnectDelay = "cassandra.connection.reconnect.delay.min" 178 | val DefaultMinReconnectDelay = 1000 179 | 180 | /** Maximum period of time, in ms, to wait before reconnecting to a dead node. */ 181 | val ConnectionMaxReconnectDelay = "cassandra.connection.reconnect.delay.max" 182 | val DefaultMaxReconnectDelay = 60000 183 | 184 | /** Compression to use (LZ4, SNAPPY or NONE). */ 185 | val ConnectionCompression = "cassandra.connection.compression" 186 | val DefaultCompression = ProtocolOptions.Compression.NONE 187 | 188 | /** The default consistency level if not explicitly passed in to a task is QUORUM. 189 | * NOTE: see this link for the string values: 190 | * http://docs.datastax.com/en/drivers/java/2.1/com/datastax/driver/core/ConsistencyLevel.html */ 191 | val ConnectionConsistency = "cassandra.connection.consistency" 192 | val DefaultConsistency = ConsistencyLevel.QUORUM 193 | 194 | private[cassandra] def hosts(s: String): Seq[InetAddress] = 195 | for { 196 | name <- s.split(",").toSeq 197 | address <- Try(InetAddress.getByName(name.trim)).toOption 198 | } yield address 199 | } 200 | 201 | trait ConnectionValidation extends Logging { 202 | 203 | import CassandraCluster._ 204 | 205 | def compressionV(config: Map[String, String]): ProtocolOptions.Compression = { 206 | import CompressorStrategies._ 207 | 208 | val valid = (a: String) => 209 | a == DeflateCompressor || a == LZ4Compressor || a == SnappyCompressor 210 | 211 | config.get(ConnectionCompression) match { 212 | case Some(s) if valid(s) => 213 | ProtocolOptions.Compression.valueOf(s) 214 | case Some(invalid) => 215 | logger.warn( 216 | s"""Configured compression type must be valid but 217 | found '$invalid'. Using the default.""") 218 | DefaultCompression 219 | case None => DefaultCompression 220 | } 221 | } 222 | 223 | object CompressorStrategies { 224 | val DeflateCompressor = "DeflateCompressor" 225 | val SnappyCompressor = "SnappyCompressor" 226 | val LZ4Compressor = "LZ4Compressor" 227 | } 228 | 229 | } -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/ConnectorSyntax.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import org.apache.kafka.connect.errors.ConnectException 20 | import com.datastax.driver.core._ 21 | import org.joda.time.{DateTimeZone, DateTime} 22 | 23 | object Types { 24 | 25 | /** A Kafka topic name from config. */ 26 | type TopicName = String 27 | 28 | /** A Cassandra keyspace name from config. */ 29 | type KeyspaceName = String 30 | 31 | /** A Cassandra table name from config. */ 32 | type TableName = String 33 | 34 | /** The formatted key to get topic to QueryNamespace mappings from config. */ 35 | type Key = String 36 | 37 | type Cql = String 38 | type Token = String 39 | type Clause = String 40 | type OrderClause = Clause 41 | type PredicatesClause = Clause 42 | type LimitClause = Clause 43 | type SelectedColumns = Clause 44 | type FilteringClause = Clause 45 | type ColumnName = String 46 | } 47 | 48 | /** INTERNAL API */ 49 | private[cassandra] object Syntax { 50 | 51 | final val Insert = "insert" 52 | final val Select = "select" 53 | final val From = "from" 54 | final val Where = "where" 55 | final val OrderByClause = "order by" 56 | final val ClusteringOrderBy = "clustering " + OrderBy 57 | final val Asc = "asc" 58 | final val Desc = "desc" 59 | final val And = "and" 60 | final val Limit = "limit" 61 | final val Filtering = "allow filtering" 62 | final val PreviousTime = "previousTime()" 63 | final val CurrentTime = "currentTime()" 64 | final val RpcAddressColumn = "rpc_address" 65 | final val ListenAddressColumn = "peer" 66 | final val SystemLocal = "system.local" 67 | final val SystemPeers = "system.peers" 68 | final val EmptyClause = "" 69 | def quote(name: String): String = "\"" + name + "\"" 70 | 71 | 72 | import Types._,InternalConfig.Schema 73 | 74 | trait Query { 75 | def cql: Cql 76 | } 77 | 78 | /** The provided user's insert statement as prepared statement query. 79 | * Used prior to `cql` if conditions amount to increased efficiency. 80 | * Not simple to infer from a collection of kafka `SinkRecord`s 81 | * holding just the topic, so a few best efforts are made. 82 | */ 83 | final case class PreparedQuery private(cql: Cql) extends Query 84 | 85 | object PreparedQuery { 86 | def apply(schema: Schema): PreparedQuery = { 87 | val columns = schema.columnNames.mkString(",") 88 | val bind = schema.columnNames.map(v => "?").mkString(",") 89 | PreparedQuery(s"INSERT INTO ${schema.namespace}($columns) VALUES($bind)") 90 | } 91 | } 92 | 93 | /** Cassandra table writes for a Kafka Connect task. 94 | * 95 | * Example: 96 | * {{{ 97 | * INSERT INTO $namespace($columns) VALUES($columnValues) 98 | * }}} 99 | * 100 | * @param cql the provided user's insert query 101 | */ 102 | final case class SinkQuery private(cql: Cql) extends Query 103 | 104 | object SinkQuery { 105 | 106 | def valid(namespace: String): Boolean = { 107 | namespace.length >= 3 || namespace.contains(".") 108 | } 109 | 110 | def apply(namespace: String, columnNames: List[ColumnName], columnValues: String): SinkQuery = { 111 | val columns = columnNames.mkString(",") 112 | SinkQuery(s"INSERT INTO $namespace($columns) VALUES($columnValues)") 113 | } 114 | } 115 | 116 | /** Cassandra table reads for a Kafka Connect task. 117 | * 118 | * Example: 119 | * {{{ 120 | * SELECT $columns FROM $keyspace.$table WHERE $filter $orderBy $limit ALLOW FILTERING 121 | * }}} 122 | * 123 | * @param cql the provided user's select query 124 | */ 125 | final case class SourceQuery private(cql: String, 126 | primaryKeys: List[ColumnName], 127 | pollInterval: Long, 128 | utc: Boolean) extends Query with QueryParser { 129 | 130 | /** WIP and Temporary (until Cassandra CDC ticket): 131 | * 132 | * It is difficult to infer timeseries from a CQL query, as it depends on the data model. 133 | * Until CDC: we make a best effort. 134 | */ 135 | def slide: SourceQuery = cql match { 136 | case query if hasPatternT => 137 | val now = if (utc) new DateTime(DateTimeZone.UTC).getMillis else System.nanoTime 138 | copy(cql = cql 139 | .replaceAllLiterally(PreviousTime, s"${now - pollInterval}") 140 | .replaceAllLiterally(CurrentTime, s"$now")) 141 | case query if hasRange(primaryKeys) => 142 | //TODO move range tokens https://tuplejump.atlassian.net/browse/DB-56 143 | this 144 | case _ => 145 | this 146 | } 147 | } 148 | 149 | object SourceQuery { 150 | 151 | def valid(cql: Cql): Boolean = { 152 | val query = cql.toLowerCase 153 | query.startsWith(Syntax.Select) && query.contains(Syntax.From) 154 | } 155 | 156 | /** Returns a new `SourceQuery` if params are valid. 157 | * 158 | * @param query the user's query where `cql` might look like: 159 | * {{{ 160 | * SELECT $columns FROM $keyspace.$table WHERE $filter $orderBy $limit ALLOW FILTERING 161 | * }}} 162 | * @param schema the schema mapping 163 | * @param interval the poll interval 164 | * @param utc `true` for UTC time zone on prev/current time slides, `false` for system time 165 | */ 166 | def apply(query: Cql, schema: Schema, interval: Long, utc: Boolean): Option[SourceQuery] = { 167 | if (valid(query)) Some(SourceQuery(query, schema.primaryKeys, interval, utc)) else None 168 | } 169 | } 170 | 171 | trait QueryParser { 172 | 173 | def cql: Cql 174 | 175 | def hasPatternT: Boolean = cql.contains(PreviousTime) || cql.contains(CurrentTime) 176 | 177 | /** Attempts to detect a very simple range query. 178 | * 179 | * {{{ 180 | * CREATE TABLE fu ( 181 | * key text, 182 | * detail_key int, 183 | * detail_value text, 184 | * PRIMARY KEY (key, detail_key) 185 | * ) 186 | * }}} 187 | * 188 | * In this case we detect the primary key, `detail_key`, is used as a range: 189 | * {{{ 190 | * select * from fu where key = '1' and detail_key > 2 and detail_key <=4; 191 | * }}} 192 | * 193 | * Other sample range query formats: 194 | * {{{ 195 | * SELECT * FROM test WHERE token(k) > token(42); 196 | * SELECT * FROM test WHERE fu >= $startToken AND fu < $endToken; 197 | * }}} 198 | */ 199 | def hasRange(primaryKeys: List[ColumnName]): Boolean = 200 | cql.split(Where).lastOption.exists(hasSimpleRange(_,primaryKeys)) 201 | 202 | def hasSimpleRange(clause: Clause, primaryKeys: List[ColumnName]): Boolean = 203 | clause.split(" ") 204 | .filter(c => primaryKeys contains c) 205 | .groupBy(x => x) 206 | .mapValues(_.length) 207 | .exists(_._2 > 1) // is repeated 208 | } 209 | 210 | /** A single token from: 211 | * {{{ 212 | * WHERE s"$token >= $startToken 213 | * AND $token < $endToken" 214 | * }}} 215 | * would be {{{ s"$token >= $startToken" }}}. An instance of a `WhereClause` 216 | * holds any WHERE predicates for use. 217 | */ 218 | final case class WhereClause(predicates: Seq[String], values: Seq[Any]) 219 | 220 | object WhereClause { 221 | val AllRows = WhereClause(Nil, Nil) 222 | } 223 | 224 | sealed trait OrderBy 225 | object OrderBy { 226 | case object Ascending extends OrderBy 227 | case object Descending extends OrderBy 228 | 229 | def apply(cql: Cql): OrderBy = 230 | if (cql.contains(Syntax.Asc)) Ascending 231 | else Descending 232 | } 233 | 234 | sealed trait Order { 235 | def by: OrderBy 236 | } 237 | 238 | /* CLUSTERING ORDER BY */ 239 | final case class ClusteringOrder(by: OrderBy) extends Order 240 | 241 | object ClusteringOrder { 242 | def apply(table: TableMetadata): Option[ClusteringOrder] = 243 | if (table.hasClusteringOrder) Some(ClusteringOrder(OrderBy(table.cql))) 244 | else None 245 | } 246 | } 247 | 248 | 249 | /** INTERNAL API */ 250 | private[cassandra] trait CassandraClusterApi { 251 | import InternalConfig.Route 252 | import Syntax._ 253 | 254 | def session: Session 255 | 256 | protected lazy val metadata = session.getCluster.getMetadata 257 | 258 | /** Returns table metadata if the keyspace and table exist in the Cassandra cluster 259 | * being connected to, and the coordinates have been configured. 260 | * 261 | * @throws org.apache.kafka.connect.errors.ConnectException 262 | * The datastax java driver returns `null` Keyspace or `null` Table 263 | * if either do not exist, so we alert the user application via 264 | * ConnectException and do not proceed, to not propagate NPEs. 265 | */ 266 | protected def tableFor(ns: Route): Option[TableMetadata] = 267 | for { 268 | keyspace <- Option(metadata.getKeyspace(ns.keyspace)).orElse(throw new ConnectException( 269 | s"Keyspace '${ns.keyspace}' does not exist.")) 270 | table <- Option(keyspace.getTable(ns.table)).orElse(throw new ConnectException( 271 | s"Table '${ns.table}' in keyspace '${ns.keyspace}' does not exist.")) 272 | } yield table 273 | 274 | /** Returns the FQCN of the partitioner, which will NOT be on the classpath. 275 | * org.apache.cassandra.dht.{Murmur3Partitioner, RandomPartitioner...} 276 | */ 277 | protected def partitioner: String = { 278 | session.execute(s"SELECT partitioner FROM $SystemLocal").one().getString(0) 279 | } 280 | 281 | // TODO: refactor when CASSANDRA-9436 282 | protected def localNode(rpcAddressColumn: String, listenAddressColumn: String): ResultSet = 283 | session.execute(s"SELECT $rpcAddressColumn, $listenAddressColumn FROM $SystemPeers") 284 | } 285 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "{}" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright {yyyy} {name of copyright owner} 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | 203 | -------------------------------------------------------------------------------- /src/main/scala/com/tuplejump/kafka/connect/cassandra/TaskConfig.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Tuplejump 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.tuplejump.kafka.connect.cassandra 18 | 19 | import scala.collection.immutable 20 | import scala.util.control.NonFatal 21 | import org.apache.kafka.common.config.ConfigException 22 | import org.apache.kafka.connect.connector.Task 23 | import com.datastax.driver.core.{TableMetadata, ConsistencyLevel} 24 | import InternalConfig._ 25 | 26 | /** A [[CassandraSourceTask]] will have [[TaskConfig.source]] list. 27 | * 28 | * A [[CassandraSinkTask]] will have [[TaskConfig.sink]] list. 29 | */ 30 | final class TaskConfig(val source: immutable.List[SourceConfig], 31 | val sink: immutable.List[SinkConfig]) 32 | 33 | object TaskConfig { 34 | import Types._ 35 | 36 | final val Empty = new TaskConfig(Nil, Nil) 37 | 38 | def apply(routes: Map[Route, Option[TableMetadata]], 39 | config: Map[String,String], 40 | taskClass: Class[_ <: Task]): TaskConfig = { 41 | 42 | val schemas = for { 43 | (route, optT) <- routes.toList 44 | schema <- Schema(route, optT) 45 | } yield schema 46 | 47 | apply(schemas, config, taskClass) 48 | } 49 | 50 | def apply(schemas: List[Schema], config: Map[String,String], taskClass: Class[_ <: Task]): TaskConfig = { 51 | configRequire(schemas.nonEmpty, s"No schemas were created - invalid configuration.") 52 | 53 | taskClass match { 54 | case `source` => 55 | val sources = for { 56 | schema <- schemas 57 | query <- schema.route.query 58 | sc <- SourceConfig(config, query, schema) 59 | } yield sc 60 | 61 | configRequire(sources.nonEmpty, s"No SourceConfigs created - invalid configuration: $config") 62 | new TaskConfig(sources, Nil) 63 | case `sink` => 64 | val sinks = for { 65 | schema <- schemas 66 | } yield SinkConfig(config, schema) 67 | 68 | configRequire(sinks.nonEmpty, s"No SinkConfigs created - invalid configuration: $config") 69 | new TaskConfig(Nil, sinks) 70 | case _ => TaskConfig.Empty 71 | } 72 | } 73 | 74 | /* **** Route config **** */ 75 | final val TopicSeparator = "," 76 | final val RouteSeparator = "." 77 | 78 | /* **** Source configurations **** */ 79 | final val SourceNamespace = "cassandra.source" 80 | 81 | /** SourceRoute + your.topic -> query */ 82 | final val SourceRoute: Key = "cassandra.source.route." 83 | 84 | /** Frequency in ms to poll for new data in each table. */ 85 | final val SourcePollInterval: Key = "cassandra.source.poll.interval" 86 | final val DefaultPollInterval: Long = 60 * 1000 87 | 88 | /** Size of Cassandra data to be read in a single Kafka task; 89 | * determines the number of partitions. */ 90 | final val SourceSplitSize: Key = "cassandra.source.split.size" 91 | final val DefaultSplitSize = 64 92 | 93 | /** Number of CQL rows to fetch in a single round-trip to Cassandra. */ 94 | final val SourceFetchSize: Key = "cassandra.source.fetch.size" 95 | final val DefaultFetchSize = 1000 96 | 97 | /** Consistency level for reads, defaults to LOCAL_ONE; 98 | * higher consistency level will disable data-locality. */ 99 | final val SourceConsistency: Key = "cassandra.source.consistency" 100 | final val DefaultSourceConsistency = ConsistencyLevel.LOCAL_ONE 101 | 102 | final val SourceUtcTimezone: Key = "cassandra.source.timezone" 103 | final val DefaultUtcTimezone = true 104 | 105 | /* **** Sink configurations **** */ 106 | final val SinkNamespace = "cassandra.sink" 107 | 108 | /** For mapping each configured Kafka topic to the Cassandra `keyspace.table` in order 109 | * to create the mapping. 110 | * 111 | * To write events from a 'devices.user.events' Kafka topic's event stream to a 112 | * timeseries Cassandra cluster's keyspace 'devices_timeseries' and table 'device_events': 113 | * Kafka devices.user.events => Cassandra devices_timeseries.user_events. 114 | * then the value of devices.user.events.table.sink should be devices_timeseries.user_events. 115 | * {{{ 116 | * Map(SinkRoute + "devices.user.events" -> "timeseries_keyspace.user_events", 117 | * SinkRoute + "devices.sensor.events" -> "timeseries_keyspace.sensor_events") 118 | * }}} 119 | */ 120 | final val SinkRoute: Key = "cassandra.sink.route." 121 | 122 | /** The consistency level for writes to Cassandra. */ 123 | final val SinkConsistency: Key = "cassandra.sink.consistency" 124 | final val DefaultSinkConsistency = ConsistencyLevel.LOCAL_QUORUM 125 | 126 | /* **** Task config **** */ 127 | final val TaskParallelismStrategy: Key = "cassandra.task.parallelism" 128 | 129 | sealed trait TaskStrategy { 130 | def key: Key 131 | } 132 | object TaskStrategy { 133 | /** One topic-to-keyspace.table per task. */ 134 | case object OneToOne extends TaskStrategy { 135 | def key: Key = "one-to-one" 136 | } 137 | /** N-topic-to-keyspace.table per task. */ 138 | case object OneToMany extends TaskStrategy { 139 | def key: Key = "one-to-many" 140 | } 141 | 142 | def apply(config: immutable.Map[String, String]): TaskStrategy = 143 | config.valueNonEmpty(TaskParallelismStrategy) match { 144 | case Some(strategy) if strategy == OneToMany.key => OneToMany 145 | case _ => OneToOne 146 | } 147 | } 148 | } 149 | 150 | /** INTERNAL API. */ 151 | private[cassandra] object InternalConfig { 152 | import com.datastax.driver.core.ConsistencyLevel 153 | import Types._ 154 | 155 | /* valueOr functions */ 156 | def toInt(a: String): Int = a.toInt 157 | def toLong(a: String): Long = a.toLong 158 | def toConsistency(a: String): ConsistencyLevel = ConsistencyLevel.valueOf(a) 159 | 160 | /** A Cassandra `keyspace.table` to Kafka topic mapping. 161 | * 162 | * @param keyspace the keyspace name to use for a kafka connect task 163 | * @param table the table name to use for a kafka connect task 164 | */ 165 | final case class Route private(topic: TopicName, 166 | keyspace: KeyspaceName, 167 | table: TableName, 168 | query: Option[Cql]) { 169 | 170 | def namespace: String = s"$keyspace.$table" 171 | 172 | } 173 | 174 | /** At this point `s` is known to not be empty but it must 175 | * have the expected length and format, which at a minimum is `a.b`. 176 | * 177 | * A `SourceConfig` has a `SourceQuery` such as: 178 | * {{{ 179 | * SELECT $columns FROM $keyspace.$table WHERE $filter $orderBy $limit ALLOW FILTERING 180 | * }}} 181 | * 182 | * A `SinkQuery` might look like this, based on keyspace.table: 183 | * {{{ 184 | * INSERT INTO $namespace($columns) VALUES($columnValues) 185 | * }}} 186 | */ 187 | object Route { 188 | 189 | final val NamespaceRegex = """(?i)(?:from|into)\s+([^.]+)\.(\S+)""".r 190 | 191 | /** Created for sources and sinks, the one to one mapping of Kafka topic 192 | * to Cassandra keyspace.table. 193 | * 194 | * At this point we know all key value params are not null, not empty. 195 | * 196 | * @throws org.apache.kafka.common.config.ConfigException if `value` is invalid 197 | */ 198 | def apply(topicKey: String, value: String): Option[Route] = { 199 | 200 | if (topicKey.contains(".route.") && value.contains(".")) { 201 | val msg = (topic: String) => s"'$value' for '$topic' must be valid and contain: keyspace.table" 202 | 203 | (topicKey.trim, value.toLowerCase) match { 204 | case (k,v) if k startsWith TaskConfig.SourceRoute => 205 | val topic = k.replace(TaskConfig.SourceRoute, "") 206 | try { 207 | (for (a <- (NamespaceRegex findAllIn v).matchData) yield a match { 208 | case NamespaceRegex(x: String, y: String) => 209 | create(topic, Option(x), Option(y), Some(value)) 210 | case _ => 211 | throw new ConfigException(msg(topic)) 212 | }).flatten.toSeq.headOption //clean up 213 | } catch { case NonFatal(e) => throw new ConfigException(msg(topic)) } 214 | 215 | case (k,v) => 216 | val topic = k.replace(TaskConfig.SinkRoute, "") 217 | configRequire(v.length >= 3, msg(topic)) 218 | val split = v.split("\\.").toList 219 | configRequire(split.size == 2, msg(topic)) 220 | create(topic, split.headOption, split.lastOption, None) 221 | } 222 | } else None 223 | } 224 | 225 | private def create(topic: TopicName, 226 | keyspace: Option[KeyspaceName], 227 | table: Option[TableName], 228 | query: Option[Cql]): Option[Route] = { 229 | //the one place topic is validated for all cases, notify the user if invalid vs return None in for comp 230 | configRequire(Option(topic).forall(_.nonEmpty), "The topic must not be empty.") 231 | 232 | for { 233 | a <- keyspace.filter(_.nonEmpty) 234 | b <- table.filter(_.nonEmpty) 235 | } yield Route(topic, a, b, query) 236 | } 237 | } 238 | 239 | import org.apache.kafka.connect.sink.SinkRecord 240 | import Types._, Syntax._,TaskConfig._ 241 | 242 | /** Note that the `cql` is the CREATE cql statement for the table metadata. 243 | * We can use this to parse options. 244 | */ 245 | final case class Schema(route: Route, 246 | partitionKey: List[ColumnName], 247 | primaryKeys: List[ColumnName], 248 | clusteringColumns: List[ColumnName], 249 | columnNames: List[ColumnName], 250 | cql: Cql) { 251 | 252 | def namespace: String = route.namespace 253 | def quotedValue: String = s"${quote(route.keyspace)}.${quote(route.table)}" 254 | 255 | /** Returns true if record and schema have same topic and fields. */ 256 | def is(record: SinkRecord): Boolean = { 257 | val comparable = (list: List[String]) => list.map(_.toLowerCase) 258 | 259 | record.topic == route.topic && comparable(columnNames) == comparable(record.asColumnNames) 260 | } 261 | } 262 | 263 | object Schema { 264 | 265 | /** Returns Kafka topic to Cassandra keyspace and table metadata 266 | * if the keyspace and table exist in the Cassandra cluster 267 | * being connected to, and the coordinates have been configured. 268 | */ 269 | def apply(route: Route, table: Option[TableMetadata]): Option[Schema] = 270 | for (t <- table) yield Schema( 271 | route, 272 | partitionKey = t.partitionKeyNames, 273 | primaryKeys = t.primaryKeyNames, 274 | clusteringColumns = t.clusteringColumnNames, 275 | columnNames = t.columnNames, 276 | cql = t.cql) 277 | } 278 | 279 | trait CassandraConfig { 280 | def schema: Schema 281 | def query: Query 282 | def options: ClusterQueryOptions 283 | } 284 | 285 | /** A Kafka Connect [[CassandraSource]] and [[CassandraSourceTask]] configuration. 286 | * INTERNAL API. 287 | */ 288 | final case class SourceConfig(schema: Schema, 289 | query: SourceQuery, 290 | options: ReadOptions) extends CassandraConfig 291 | 292 | object SourceConfig { 293 | 294 | def apply(config: Map[String,String], cql: Cql, schema: Schema): Option[SourceConfig] = 295 | for { 296 | query <- SourceQuery(cql, schema, 297 | config.valueOr[Long](SourcePollInterval, toLong, DefaultPollInterval), 298 | DefaultUtcTimezone) 299 | } yield SourceConfig(schema, query, ReadOptions(config)) 300 | } 301 | 302 | /** A Kafka Connect [[CassandraSink]] and [[CassandraSinkTask]] configuration. 303 | * 304 | * @param schema the kafka `topic` mapping to a cassandra keyspace and table, 305 | * with the table schema 306 | */ 307 | final case class SinkConfig(schema: Schema, 308 | query: PreparedQuery, 309 | options: WriteOptions) extends CassandraConfig 310 | 311 | /* TODO TTL, Timestamp, other write settings. Or, user pass in their insert query. */ 312 | object SinkConfig { 313 | import Syntax._ 314 | 315 | def apply(config: Map[String, String], schema: Schema): SinkConfig = 316 | SinkConfig(schema, PreparedQuery(schema), WriteOptions(config)) 317 | } 318 | 319 | sealed trait ClusterQueryOptions 320 | 321 | /** Settings related for individual queries, can be set per keyspace.table. */ 322 | final case class WriteOptions(consistency: ConsistencyLevel) extends ClusterQueryOptions 323 | 324 | object WriteOptions { 325 | 326 | def apply(config: Map[String,String]): WriteOptions = 327 | WriteOptions(config.valueOr[ConsistencyLevel]( 328 | SinkConsistency, toConsistency, DefaultSourceConsistency)) 329 | } 330 | 331 | /** Settings related for individual queries, can be set per keyspace.table. */ 332 | final case class ReadOptions(splitSize: Int, 333 | fetchSize: Int, 334 | consistency: ConsistencyLevel, 335 | limit: Option[Long]) extends ClusterQueryOptions 336 | 337 | object ReadOptions { 338 | 339 | def apply(config: Map[String,String]): ReadOptions = 340 | ReadOptions( 341 | splitSize = config.valueOr[Int](SourceSplitSize, toInt, DefaultSplitSize), 342 | fetchSize = config.valueOr[Int](SourceFetchSize, toInt, DefaultFetchSize), 343 | consistency = config.valueOr[ConsistencyLevel]( 344 | SourceConsistency, toConsistency, DefaultSourceConsistency), 345 | None) 346 | } 347 | } 348 | --------------------------------------------------------------------------------