├── 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 [](https://github.com/tuplejump/kafka-connect-cassandra/blob/master/LICENSE) [](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 |
--------------------------------------------------------------------------------