In case of experiencing the following error: {@code Error while sending value. 43 | * com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout during write query 44 | * at consistency LOCAL_ONE (1 replica were required but only 0 acknowledged the write)}, 45 | * 46 | *
it is recommended to increase the Cassandra write timeout to adapt to your workload in your
47 | * Cassandra cluster so that such timeout errors do not happen. For that you need to raise
48 | * write_request_timeout_in_ms conf parameter in your cassandra.yml. Indeed, This exception means
49 | * that Cassandra coordinator node (internal Cassandra) waited too long for an internal replication
50 | * (replication to another node and did not ack the write. It is not recommended to lower the
51 | * replication factor in your Cassandra cluster because it is mandatory that you do not loose data
52 | * in case of a Cassandra cluster failure. Waiting for a single replica for write acknowledge is the
53 | * minimum level for this guarantee in Cassandra.}
54 | *
55 | * @param Entries are in the form |operator_id | subtask_id | last_completed_checkpoint|
38 | */
39 | public class CassandraCommitter extends CheckpointCommitter {
40 |
41 | private static final long serialVersionUID = 1L;
42 |
43 | private final ClusterBuilder builder;
44 | private transient Cluster cluster;
45 | private transient Session session;
46 |
47 | private String keySpace = "flink_auxiliary";
48 | private String table = "checkpoints_";
49 |
50 | /**
51 | * A cache of the last committed checkpoint ids per subtask index. This is used to avoid
52 | * redundant round-trips to Cassandra (see {@link #isCheckpointCommitted(int, long)}.
53 | */
54 | private final Map Example:
30 | *
31 | * The above example will let the sink ignore the WriteTimeoutException, without failing the
49 | * sink. For all other failures, the sink will fail.
50 | */
51 | @PublicEvolving
52 | public interface CassandraFailureHandler extends Serializable {
53 |
54 | /**
55 | * Handle a failed {@link Throwable}.
56 | *
57 | * @param failure the cause of failure
58 | * @throws IOException if the sink should fail on this failure, the implementation should
59 | * rethrow the throwable or a custom one
60 | */
61 | void onFailure(Throwable failure) throws IOException;
62 | }
63 |
--------------------------------------------------------------------------------
/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraPojoSink.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.cassandra;
19 |
20 | import org.apache.flink.configuration.Configuration;
21 |
22 | import com.datastax.driver.core.ResultSet;
23 | import com.datastax.driver.core.Session;
24 | import com.datastax.driver.mapping.Mapper;
25 | import com.datastax.driver.mapping.MappingManager;
26 | import com.google.common.util.concurrent.ListenableFuture;
27 |
28 | import javax.annotation.Nullable;
29 |
30 | /**
31 | * Flink Sink to save data into a Cassandra cluster using Mapper,
33 | * which it uses annotations from
35 | * com.datastax.driver.mapping.annotations. Please read the recommendations in {@linkplain
36 | * CassandraSinkBase}.
37 | *
38 | * @param Note that this option is only available if using {@link ProtocolVersion#V2} or above.
42 | *
43 | * @param ttl the TTL (in seconds).
44 | */
45 | public SimpleMapperOptions ttl(int ttl) {
46 | options.add(Mapper.Option.ttl(ttl));
47 | return this;
48 | }
49 |
50 | /**
51 | * Adds a timestamp option to a mapper operation. This is only valid for save and delete
52 | * operations.
53 | *
54 | * Note that this option is only available if using {@link ProtocolVersion#V2} or above.
55 | *
56 | * @param timestamp the timestamp (in microseconds).
57 | */
58 | public SimpleMapperOptions timestamp(long timestamp) {
59 | options.add(Mapper.Option.timestamp(timestamp));
60 | return this;
61 | }
62 |
63 | /**
64 | * Adds a consistency level value option to a mapper operation. This is valid for save, delete
65 | * and get operations.
66 | *
67 | * Note that the consistency level can also be defined at the mapper level, as a parameter of
68 | * the {@link com.datastax.driver.mapping.annotations.Table} annotation (this is redundant for
69 | * backward compatibility). This option, whether defined on a specific call or as the default,
70 | * will always take precedence over the annotation.
71 | *
72 | * @param cl the {@link com.datastax.driver.core.ConsistencyLevel} to use for the operation.
73 | */
74 | public SimpleMapperOptions consistencyLevel(ConsistencyLevel cl) {
75 | options.add(Mapper.Option.consistencyLevel(cl));
76 | return this;
77 | }
78 |
79 | /**
80 | * Enables query tracing for a mapper operation. This is valid for save, delete and get
81 | * operations.
82 | *
83 | * @param enabled whether to enable tracing.
84 | */
85 | public SimpleMapperOptions tracing(boolean enabled) {
86 | options.add(Mapper.Option.tracing(enabled));
87 | return this;
88 | }
89 |
90 | /**
91 | * Specifies whether null entity fields should be included in insert queries. This option is
92 | * valid only for save operations.
93 | *
94 | * If this option is not specified, it defaults to {@code true} (null fields are saved).
95 | *
96 | * @param enabled whether to include null fields in queries.
97 | */
98 | public SimpleMapperOptions saveNullFields(boolean enabled) {
99 | options.add(Mapper.Option.saveNullFields(enabled));
100 | return this;
101 | }
102 |
103 | /**
104 | * Specifies whether an IF NOT EXISTS clause should be included in insert queries. This option
105 | * is valid only for save operations.
106 | *
107 | * If this option is not specified, it defaults to {@code false} (IF NOT EXISTS statements
108 | * are not used).
109 | *
110 | * @param enabled whether to include an IF NOT EXISTS clause in queries.
111 | */
112 | public SimpleMapperOptions ifNotExists(boolean enabled) {
113 | options.add(Mapper.Option.ifNotExists(enabled));
114 | return this;
115 | }
116 |
117 | @Override
118 | public Mapper.Option[] getMapperOptions() {
119 | return options.toArray(new Mapper.Option[0]);
120 | }
121 | }
122 |
--------------------------------------------------------------------------------
/flink-connector-cassandra/src/main/resources/META-INF/NOTICE:
--------------------------------------------------------------------------------
1 | flink-connector-cassandra
2 | Copyright 2014-2024 The Apache Software Foundation
3 |
4 | This product includes software developed at
5 | The Apache Software Foundation (http://www.apache.org/).
6 |
7 | This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
8 |
9 | - com.datastax.cassandra:cassandra-driver-core:shaded:3.11.2
10 | - com.datastax.cassandra:cassandra-driver-mapping:3.11.2
11 | - com.google.guava:guava:19.0
12 |
--------------------------------------------------------------------------------
/flink-connector-cassandra/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF 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 org.apache.flink.architecture;
20 |
21 | import org.apache.flink.architecture.common.ImportOptions;
22 |
23 | import com.tngtech.archunit.core.importer.ImportOption;
24 | import com.tngtech.archunit.junit.AnalyzeClasses;
25 | import com.tngtech.archunit.junit.ArchTest;
26 | import com.tngtech.archunit.junit.ArchTests;
27 |
28 | /** product code Architecture tests. */
29 | @AnalyzeClasses(
30 | packages = "org.apache.flink.connector",
31 | importOptions = {
32 | ImportOption.DoNotIncludeTests.class,
33 | ImportOption.DoNotIncludeArchives.class,
34 | ImportOptions.ExcludeScalaImportOption.class,
35 | ImportOptions.ExcludeShadedImportOption.class
36 | })
37 | public class ProductionCodeArchitectureTest {
38 |
39 | @ArchTest
40 | public static final ArchTests COMMON_TESTS = ArchTests.in(ProductionCodeArchitectureBase.class);
41 | }
42 |
--------------------------------------------------------------------------------
/flink-connector-cassandra/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF 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 org.apache.flink.architecture;
20 |
21 | import org.apache.flink.architecture.common.ImportOptions;
22 |
23 | import com.tngtech.archunit.core.importer.ImportOption;
24 | import com.tngtech.archunit.junit.AnalyzeClasses;
25 | import com.tngtech.archunit.junit.ArchTest;
26 | import com.tngtech.archunit.junit.ArchTests;
27 |
28 | /** Architecture tests for test code. */
29 | @AnalyzeClasses(
30 | packages = {
31 | "org.apache.flink.batch.connectors.cassandra",
32 | "org.apache.flink.streaming.connectors.cassandra",
33 | "org.apache.flink.connector.cassandra",
34 | "org.apache.flink.connectors.cassandra"
35 | },
36 | importOptions = {
37 | ImportOption.OnlyIncludeTests.class,
38 | ImportOptions.ExcludeScalaImportOption.class,
39 | ImportOptions.ExcludeShadedImportOption.class
40 | })
41 | public class TestCodeArchitectureTest {
42 |
43 | @ArchTest
44 | public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
45 | }
46 |
--------------------------------------------------------------------------------
/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.batch.connectors.cassandra.example;
19 |
20 | import org.apache.flink.api.common.typeinfo.TypeHint;
21 | import org.apache.flink.api.java.DataSet;
22 | import org.apache.flink.api.java.ExecutionEnvironment;
23 | import org.apache.flink.api.java.tuple.Tuple2;
24 | import org.apache.flink.api.java.typeutils.TupleTypeInfo;
25 | import org.apache.flink.batch.connectors.cassandra.CassandraInputFormat;
26 | import org.apache.flink.batch.connectors.cassandra.CassandraTupleOutputFormat;
27 | import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
28 |
29 | import com.datastax.driver.core.Cluster;
30 | import com.datastax.driver.core.Cluster.Builder;
31 |
32 | import java.util.ArrayList;
33 |
34 | /**
35 | * This is an example showing the to use the Cassandra Input-/OutputFormats in the Batch API.
36 | *
37 | * The example assumes that a table exists in a local cassandra database, according to the
38 | * following queries: CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class':
39 | * 'SimpleStrategy', 'replication_factor': '1'}; CREATE TABLE IF NOT EXISTS test.batches (number
40 | * int, strings text, PRIMARY KEY(number, strings));
41 | */
42 | public class BatchExample {
43 | private static final String INSERT_QUERY =
44 | "INSERT INTO test.batches (number, strings) VALUES (?,?);";
45 | private static final String SELECT_QUERY = "SELECT number, strings FROM test.batches;";
46 |
47 | /*
48 | * table script: "CREATE TABLE test.batches (number int, strings text, PRIMARY KEY(number, strings));"
49 | */
50 | public static void main(String[] args) throws Exception {
51 |
52 | ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
53 | env.setParallelism(1);
54 |
55 | ArrayList The example assumes that a table exists in a local cassandra database, according to the
41 | * following queries: CREATE KEYSPACE IF NOT EXISTS flink WITH replication = {'class':
42 | * 'SimpleStrategy', 'replication_factor': '1'}; CREATE TABLE IF NOT EXISTS flink.batches (id text,
43 | * counter int, batch_id int, PRIMARY KEY(id, counter, batchId));
44 | */
45 | public class BatchPojoExample {
46 | private static final String SELECT_QUERY = "SELECT id, counter, batch_id FROM flink.batches;";
47 |
48 | public static void main(String[] args) throws Exception {
49 |
50 | ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
51 | env.setParallelism(1);
52 |
53 | List Pojo's have to be annotated with datastax annotations to work with this sink.
35 | *
36 | * The example assumes that a table exists in a local cassandra database, according to the
37 | * following queries: CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class':
38 | * 'SimpleStrategy', 'replication_factor': '1'}; CREATE TABLE IF NOT EXISTS test.message(body txt
39 | * PRIMARY KEY)
40 | */
41 | public class CassandraPojoSinkExample {
42 | private static final ArrayList The example assumes that a table exists in a local cassandra database, according to the
35 | * following queries: CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class':
36 | * 'SimpleStrategy', 'replication_factor': '1'}; CREATE TABLE IF NOT EXISTS test.writetuple(element1
37 | * text PRIMARY KEY, element2 int)
38 | */
39 | public class CassandraTupleSinkExample {
40 | private static final String INSERT =
41 | "INSERT INTO test.writetuple (element1, element2) VALUES (?, ?)";
42 | private static final ArrayList The example assumes that a table exists in a local cassandra database, according to the
41 | * following queries: CREATE KEYSPACE IF NOT EXISTS example WITH replication = {'class':
42 | * 'SimpleStrategy', 'replication_factor': '1'}; CREATE TABLE example.values (id text, count int,
43 | * PRIMARY KEY(id));
44 | *
45 | * Important things to note are that checkpointing is enabled, a StateBackend is set and the
46 | * enableWriteAheadLog() call when creating the CassandraSink.
47 | */
48 | public class CassandraTupleWriteAheadSinkExample {
49 | public static void main(String[] args) throws Exception {
50 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
51 | env.setParallelism(1);
52 | env.enableCheckpointing(1000);
53 | env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));
54 | env.setStateBackend(
55 | new FsStateBackend(
56 | "file:///" + System.getProperty("java.io.tmpdir") + "/flink/backend"));
57 |
58 | CassandraSink{@code
32 | * private static class ExampleFailureHandler implements CassandraFailureHandler {
33 | *
34 | * @Override
35 | * void onFailure(Throwable failure) throws IOException {
36 | * if (ExceptionUtils.findThrowable(failure, WriteTimeoutException.class).isPresent()) {
37 | * // drop exception
38 | * } else {
39 | * // for all other failures, fail the sink;
40 | * // here the failure is simply rethrown, but users can also choose to throw custom exceptions
41 | * throw failure;
42 | * }
43 | * }
44 | * }
45 | *
46 | * }
47 | *
48 | *