diff --git a/flink-connector-cassandra/archunit-violations/dcfaa83d-a12c-48e1-9e51-b8d3808cd287 b/flink-connector-cassandra/archunit-violations/dcfaa83d-a12c-48e1-9e51-b8d3808cd287 index 3fc516c..9970853 100644 --- a/flink-connector-cassandra/archunit-violations/dcfaa83d-a12c-48e1-9e51-b8d3808cd287 +++ b/flink-connector-cassandra/archunit-violations/dcfaa83d-a12c-48e1-9e51-b8d3808cd287 @@ -1,14 +1,86 @@ +Constructor (org.apache.flink.connector.cassandra.sink.config.CassandraSinkConfig)> calls method in (CassandraSinkBuilder.java:122) +Constructor (org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration, org.apache.flink.connector.cassandra.sink.config.CassandraSinkConfig, org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, org.apache.flink.connector.cassandra.sink.exception.CassandraFailureHandler, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.connector.cassandra.sink.CassandraSinkWriter$RecordWriterProvider)> calls constructor (int)> in (CassandraSinkWriter.java:142) +Constructor (org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration, org.apache.flink.connector.cassandra.sink.config.CassandraSinkConfig, org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, org.apache.flink.connector.cassandra.sink.exception.CassandraFailureHandler, org.apache.flink.api.common.operators.MailboxExecutor, org.apache.flink.metrics.groups.SinkWriterMetricGroup, org.apache.flink.connector.cassandra.sink.CassandraSinkWriter$RecordWriterProvider)> calls method in (CassandraSinkWriter.java:148) +Constructor (java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (PojoSinkConfig.java:73) +Constructor (java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (PojoSinkConfig.java:74) +Constructor (org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver, org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver, org.apache.flink.connector.cassandra.sink.planner.core.strategy.PlannerStrategy, org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver, org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer)> calls method in (StatementPlanner.java:80) +Constructor (org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver, org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver, org.apache.flink.connector.cassandra.sink.planner.core.strategy.PlannerStrategy, org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver, org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer)> calls method in (StatementPlanner.java:81) +Constructor (org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver, org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver, org.apache.flink.connector.cassandra.sink.planner.core.strategy.PlannerStrategy, org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver, org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer)> calls method in (StatementPlanner.java:83) +Constructor (java.util.List, [Ljava.lang.Object;, java.util.List, [Ljava.lang.Object;)> calls method in (ResolvedWrite.java:98) +Constructor (java.util.List, [Ljava.lang.Object;, java.util.List, [Ljava.lang.Object;)> calls method in (ResolvedWrite.java:105) +Constructor (java.util.List, [Ljava.lang.Object;, java.util.List, [Ljava.lang.Object;)> calls method in (ResolvedWrite.java:86) +Constructor (java.util.List, [Ljava.lang.Object;, java.util.List, [Ljava.lang.Object;)> calls method in (ResolvedWrite.java:91) +Constructor (java.util.List, [Ljava.lang.Object;, java.util.List, [Ljava.lang.Object;)> calls method in (ResolvedWrite.java:80) +Constructor (java.util.List, [Ljava.lang.Object;, java.util.List, [Ljava.lang.Object;)> calls method in (ResolvedWrite.java:81) +Constructor (java.util.List, [Ljava.lang.Object;, java.util.List, [Ljava.lang.Object;)> calls method in (ResolvedWrite.java:82) +Constructor (java.util.List, [Ljava.lang.Object;, java.util.List, [Ljava.lang.Object;)> calls method in (ResolvedWrite.java:83) +Constructor (java.lang.String, java.lang.String)> calls method in (TableRef.java:38) +Constructor (java.lang.String, java.lang.String)> calls method in (TableRef.java:40) +Constructor (com.datastax.driver.core.Session, org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner, org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig)> calls method in (CqlRecordWriter.java:72) +Constructor (com.datastax.driver.core.Session, org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner, org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig)> calls method in (CqlRecordWriter.java:73) +Constructor (com.datastax.driver.core.Session, org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner, org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig)> calls method in (CqlRecordWriter.java:74) +Constructor (com.datastax.driver.core.Session, org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner, org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig)> is annotated with in (CqlRecordWriter.java:0) +Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig)> calls method in (CqlRecordWriter.java:69) +Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig)> calls method in (CqlRecordWriter.java:70) +Constructor (com.datastax.driver.core.Cluster, com.datastax.driver.core.Session, com.datastax.driver.mapping.Mapper)> is annotated with in (PojoRecordWriter.java:0) +Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, org.apache.flink.connector.cassandra.sink.config.PojoSinkConfig)> calls method in (PojoRecordWriter.java:71) +Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, org.apache.flink.connector.cassandra.sink.config.PojoSinkConfig)> calls method in (PojoRecordWriter.java:72) Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (CassandraSource.java:138) +Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (CassandraSource.java:122) +Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (CassandraSource.java:123) Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (CassandraSource.java:124) -Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (CassandraSource.java:125) -Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (CassandraSource.java:126) -Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (CassandraSource.java:127) +Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, java.lang.Class, java.lang.String, org.apache.flink.streaming.connectors.cassandra.MapperOptions)> calls method in (CassandraSource.java:125) Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, org.apache.flink.connector.cassandra.source.reader.converter.CassandraRowToTypeConverter, java.lang.String)> calls method in (CassandraSource.java:190) Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, org.apache.flink.connector.cassandra.source.reader.converter.CassandraRowToTypeConverter, java.lang.String)> calls method in (CassandraSource.java:194) Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, org.apache.flink.connector.cassandra.source.reader.converter.CassandraRowToTypeConverter, java.lang.String)> calls method in (CassandraSource.java:195) Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, org.apache.flink.connector.cassandra.source.reader.converter.CassandraRowToTypeConverter, java.lang.String)> calls method in (CassandraSource.java:196) Constructor (org.apache.flink.streaming.connectors.cassandra.ClusterBuilder, long, org.apache.flink.connector.cassandra.source.reader.converter.CassandraRowToTypeConverter, java.lang.String)> calls method in (CassandraSource.java:197) Constructor (org.apache.flink.connector.cassandra.source.reader.converter.CassandraRowToTypeConverter)> calls method in (CassandraRowEmitter.java:38) +Method calls method in (CassandraSinkBuilder.java:221) +Method calls method in (CassandraSinkBuilder.java:165) +Method calls method in (CassandraSinkBuilder.java:164) +Method calls method in (CassandraSinkBuilder.java:180) +Method calls method in (CassandraSinkBuilder.java:132) +Method calls method in (CassandraSinkBuilder.java:134) +Method calls method in (CassandraSinkBuilder.java:148) +Method calls method in (CassandraSinkBuilder.java:150) +Method calls method in (CassandraSinkBuilder.java:192) +Method calls method in (DynamicPlannerAssembler.java:31) +Method calls method in (DynamicPlannerAssembler.java:33) +Method calls method in (DynamicPlannerAssembler.java:36) +Method calls method in (StaticPlannerAssembler.java:46) +Method calls method in (CqlSinkConfig.java:145) +Method calls method in (CqlSinkConfig.java:124) +Method calls method in (CqlSinkConfig.java:121) +Method calls method in (CqlSinkConfig.java:162) +Method calls method in (CqlSinkConfig.java:159) +Method calls method in (RequestConfiguration.java:204) +Method calls method in (RequestConfiguration.java:152) +Method calls method in (RequestConfiguration.java:138) +Method calls method in (RequestConfiguration.java:167) +Method calls method in (SinkPluggable.java:173) +Method calls method in (SinkPluggable.java:174) +Method calls method in (PreparedStatementCache.java:88) +Method calls method in (PreparedStatementCache.java:91) +Method calls method in (InsertStrategy.java:70) +Method calls method in (StaticInsertStrategy.java:52) +Method calls method in (StaticUpdateStrategy.java:51) +Method calls method in (UpdateStrategy.java:71) +Method calls method in (QueryParser.java:131) +Method calls method in (QueryParser.java:150) +Method calls method in (QueryParser.java:136) +Method calls method in (QueryParser.java:154) +Method calls method in (QueryParser.java:360) +Method calls method in (QueryParser.java:368) +Method calls method in (QueryParser.java:173) +Method calls method in (QueryParser.java:191) +Method calls method in (QueryParser.java:194) +Method calls method in (QueryParser.java:178) +Method calls method in (QueryParser.java:292) +Method calls method in (QueryParser.java:406) +Method calls method in (QueryParser.java:415) +Method calls method in (QueryParser.java:206) +Method calls method in (QueryParser.java:207) Method calls method in (CassandraSourceBuilder.java:153) Method calls method in (CassandraSourceBuilder.java:177) Method calls method in (CassandraSourceBuilder.java:185) diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSink.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSink.java new file mode 100644 index 0000000..1f093aa --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSink.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.connector.cassandra.sink.config.CassandraSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration; +import org.apache.flink.connector.cassandra.sink.exception.CassandraFailureHandler; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; + +import java.io.IOException; + +/** + * A generic Flink {@link Sink} implementation for writing records to Apache Cassandra. + * + *

This sink supports multiple input types including POJOs, Tuples, Rows, and Scala Products. + * + *

Use {@link CassandraSinkBuilder} to create and configure instances of this sink. + * + * @param The input record type + */ +@PublicEvolving +public class CassandraSink implements Sink { + + private final CassandraSinkConfig sinkConfig; + private final ClusterBuilder clusterBuilder; + private final CassandraFailureHandler failureHandler; + private final RequestConfiguration requestConfiguration; + + /** + * Package-private constructor - use {@link CassandraSinkBuilder} to create instances. + * + * @param sinkConfig The sink-specific configuration + * @param clusterBuilder Factory for building Cassandra cluster connections + * @param failureHandler Handler for retriable and fatal failures + * @param requestConfiguration Settings for concurrency, timeouts, and retries + */ + CassandraSink( + CassandraSinkConfig sinkConfig, + ClusterBuilder clusterBuilder, + CassandraFailureHandler failureHandler, + RequestConfiguration requestConfiguration) { + this.sinkConfig = sinkConfig; + this.clusterBuilder = clusterBuilder; + this.failureHandler = failureHandler; + this.requestConfiguration = requestConfiguration; + } + + /** + * Creates a new {@link SinkWriter} for this sink. + * + * @param context the sink context providing access to MailboxExecutor and metrics + * @return a new CassandraSinkWriter instance + * @throws IOException if writer creation fails + */ + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new CassandraSinkWriter<>( + this.requestConfiguration, + this.sinkConfig, + this.clusterBuilder, + this.failureHandler, + context.getMailboxExecutor(), + context.metricGroup()); + } + + // Package-private getters for testing + CassandraSinkConfig getSinkConfig() { + return sinkConfig; + } + + ClusterBuilder getClusterBuilder() { + return clusterBuilder; + } + + CassandraFailureHandler getFailureHandler() { + return failureHandler; + } + + RequestConfiguration getRequestConfiguration() { + return requestConfiguration; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSinkBuilder.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSinkBuilder.java new file mode 100644 index 0000000..09d3d52 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSinkBuilder.java @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.cassandra.sink.config.CassandraSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.PojoSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration; +import org.apache.flink.connector.cassandra.sink.exception.CassandraFailureHandler; +import org.apache.flink.connector.cassandra.sink.planner.core.resolution.ResolutionMode; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.util.Preconditions; + +import com.datastax.driver.core.Cluster; +import org.apache.commons.lang3.StringUtils; + +/** + * Builder for creating {@link CassandraSink} instances. + * + *

This builder provides two factory methods for creating type-specific builders: + * + *

    + *
  • {@link #newBuilder(PojoSinkConfig)} - Creates a POJO builder. + *
  • {@link #newBuilder(CqlSinkConfig)} - Creates a CQL builder with {@link + * org.apache.flink.connector.cassandra.sink.planner.SinkPluggable} support + *
+ * + *

Example usage with POJO: + * + *

{@code
+ * CassandraSink sink = CassandraSinkBuilder
+ *     .newBuilder(pojoSinkConfig)
+ *     .setClusterBuilder(clusterBuilder)
+ *     .build();
+ * }
+ * + *

Example usage with CQL (STATIC mode): + * + *

{@code
+ * CqlSinkConfig cfg = CqlSinkConfig.forRow()
+ *     .withQuery("INSERT INTO ks.tbl(id,name,age) VALUES (?,?,?)");
+ *
+ * CassandraSink sink = CassandraSinkBuilder
+ *     .newBuilder(cfg)
+ *     .setClusterBuilder(clusterBuilder)
+ *     .build();
+ * }
+ * + *

Example usage with CQL (DYNAMIC mode): + * + *

{@code
+ * CqlSinkConfig cfg = CqlSinkConfig.forRow()
+ *     .withPluggable(
+ *         SinkPluggable.builder()
+ *             .withTableResolver(myTableResolver)
+ *             .withInsertResolver(myInsertResolver)
+ *             .build());
+ *
+ * CassandraSink sink = CassandraSinkBuilder
+ *     .newBuilder(cfg)
+ *     .setClusterBuilder(clusterBuilder)
+ *     .build();
+ * }
+ */ +@PublicEvolving +public final class CassandraSinkBuilder { + + /** + * Creates a new builder for POJO sink configuration. + * + * @param config The POJO sink configuration + * @param The input record type + * @return a new POJO builder instance + */ + public static PojoBuilder newBuilder(PojoSinkConfig config) { + return new PojoBuilder<>(config); + } + + /** + * Creates a new builder for CQL sink configuration. + * + * @param config The CQL sink configuration + * @param The input record type + * @return a new CQL builder instance + */ + public static CqlBuilder newBuilder(CqlSinkConfig config) { + return new CqlBuilder<>(config); + } + + /** + * Base builder with shared configuration options for all sink types. + * + * @param The input record type + * @param The concrete builder type for fluent API + */ + public abstract static class BaseBuilder> { + protected final CassandraSinkConfig sinkConfig; + protected ClusterBuilder clusterBuilder; + protected String host; + protected Integer port; + protected CassandraFailureHandler failureHandler; + protected RequestConfiguration requestConfig; + + protected BaseBuilder(CassandraSinkConfig sinkConfig) { + this.sinkConfig = Preconditions.checkNotNull(sinkConfig, "sinkConfig must not be null"); + } + + /** + * Sets the Cassandra host to connect to. + * + * @param host the Cassandra host + * @return this builder + */ + public SELF setHost(String host) { + Preconditions.checkArgument( + !StringUtils.isEmpty(host), "host must not be null or empty"); + Preconditions.checkArgument( + this.clusterBuilder == null, + "ClusterBuilder was already set. Either setHost()/setPort() or setClusterBuilder()."); + this.host = host; + return self(); + } + + /** + * Sets the Cassandra port to connect to. + * + * @param port the Cassandra port + * @return this builder + */ + public SELF setPort(int port) { + Preconditions.checkArgument( + port > 0 && port <= 65535, "port must be between 1 and 65535"); + Preconditions.checkArgument( + this.clusterBuilder == null, + "ClusterBuilder was already set. Either setHost()/setPort() or setClusterBuilder()."); + this.port = port; + return self(); + } + + /** + * Sets the ClusterBuilder for this sink. + * + * @param clusterBuilder ClusterBuilder to configure the connection to Cassandra + * @return this builder + */ + public SELF setClusterBuilder(ClusterBuilder clusterBuilder) { + Preconditions.checkNotNull(clusterBuilder, "clusterBuilder must not be null"); + Preconditions.checkArgument( + this.host == null && this.port == null, + "Connection information was already set. Either setHost()/setPort() or setClusterBuilder()."); + this.clusterBuilder = clusterBuilder; + return self(); + } + + /** + * Sets the failure handler for this sink. + * + * @param failureHandler CassandraFailureHandler that handles failures + * @return this builder + */ + public SELF setFailureHandler(CassandraFailureHandler failureHandler) { + this.failureHandler = + Preconditions.checkNotNull(failureHandler, "failureHandler must not be null"); + return self(); + } + + /** + * Sets the request configuration using a pre-built RequestConfiguration. + * + * @param requestConfig a configured RequestConfiguration. + * @return this builder + */ + public SELF setRequestConfiguration(RequestConfiguration requestConfig) { + this.requestConfig = + Preconditions.checkNotNull(requestConfig, "requestConfig must not be null"); + return self(); + } + + /** + * Builds the configured CassandraSink. This builder can only be used once. + * + * @return a new CassandraSink instance + * @throws IllegalStateException if required configuration is missing or builder already + * used + */ + public final CassandraSink build() { + // Create ClusterBuilder from host/port if not already set + if (clusterBuilder == null) { + if (host != null) { + final String finalHost = host; + final int finalPort = (port != null) ? port : 9042; // Default Cassandra port + this.clusterBuilder = + new ClusterBuilder() { + @Override + protected Cluster buildCluster(Cluster.Builder builder) { + return builder.addContactPoint(finalHost) + .withPort(finalPort) + .build(); + } + }; + } + } + + Preconditions.checkArgument( + this.clusterBuilder != null, + "Cassandra connection information must be supplied using either setHost()/setPort() or setClusterBuilder()."); + // Default Failure Handler + if (this.failureHandler == null) { + this.failureHandler = new CassandraFailureHandler(); + } + // Default Request Configuration. + if (this.requestConfig == null) { + this.requestConfig = RequestConfiguration.builder().build(); + } + + validateSpecificConfiguration(); + return new CassandraSink<>(sinkConfig, clusterBuilder, failureHandler, requestConfig); + } + + protected abstract SELF self(); + + /** Validates builder-specific configuration before building the sink. */ + protected void validateSpecificConfiguration() {} + } + + /** + * Builder for POJO sink configurations. This builder uses DataStax Mapper annotations for all + * metadata. + * + * @param The POJO input type + */ + @PublicEvolving + public static final class PojoBuilder extends BaseBuilder> { + + PojoBuilder(PojoSinkConfig config) { + super(config); + } + + @Override + protected PojoBuilder self() { + return this; + } + } + + /** + * Builder for CQL sink configurations. This builder exposes CQL-specific pluggable options for + * dynamic behavior. + * + *

Supports two sink modes: + * + *

    + *
  • STATIC: User provides explicit INSERT/UPDATE query. + *
  • DYNAMIC: Uses pluggable resolvers for runtime table/column resolution. + *
+ * + * @param The input record type (Row, Tuple, Scala.Product etc.) + */ + @PublicEvolving + public static final class CqlBuilder extends BaseBuilder> { + private final CqlSinkConfig cqlConfig; + + CqlBuilder(CqlSinkConfig config) { + super(config); + this.cqlConfig = config; + } + + @Override + protected CqlBuilder self() { + return this; + } + + @Override + protected void validateSpecificConfiguration() { + ResolutionMode resolutionMode = cqlConfig.getResolutionMode(); + if (resolutionMode == ResolutionMode.UNSET) { + throw new IllegalStateException( + "CQL sink configuration requires exactly one mode: either call withQuery(...) for STATIC mode, or withPluggable(...) for DYNAMIC mode."); + } + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriter.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriter.java new file mode 100644 index 0000000..07a8046 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriter.java @@ -0,0 +1,348 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.java.ClosureCleaner; +import org.apache.flink.connector.cassandra.sink.config.CassandraSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration; +import org.apache.flink.connector.cassandra.sink.exception.CassandraFailureHandler; +import org.apache.flink.connector.cassandra.sink.exception.MaxRetriesExceededException; +import org.apache.flink.connector.cassandra.sink.util.RecordWriterFactory; +import org.apache.flink.connector.cassandra.sink.writer.CassandraRecordWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.DescriptiveStatisticsHistogram; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.util.function.ThrowingRunnable; + +import com.datastax.driver.core.ResultSet; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * A {@link SinkWriter} implementation for Cassandra that handles async write operations with + * Flink's MailboxExecutor. + * + *

This writer provides: + * + *

    + *
  • Semaphore-based backpressure control to limit concurrent requests + *
  • Retry logic with configurable maximum attempts + *
  • Fatal vs retryable exception classification + *
  • Comprehensive metrics (success, failure, retry counts, latency) + *
+ * + *

Error Handling Contract

+ * + *

This sink writer handles different types of failures according to the following contract: + * + *

    + *
  • Permit acquisition timeout (backpressure): write() throws IOException + * immediately (synchronous). Flink's restart strategy applies for at-least-once delivery. + *
  • Retriable driver errors (WriteTimeoutException, UnavailableException, + * NoHostAvailableException, OperationTimedOutException): Retries up to maxRetries. Success → + * counted as success; exhausted → handled as max retries exhausted. + *
  • Max retries exhausted: Default CassandraFailureHandler throws IOException + * → task fails (at-least-once). Users can override to swallow for best-effort. + *
  • Fatal/config errors (Authn/Authz, InvalidQueryException, SyntaxError, + * codec/driver config issues): Immediate IOException (no retry) for at-least-once delivery. + *
+ * + * @param The input record type + */ +@Internal +class CassandraSinkWriter implements SinkWriter { + + private static final int PERMITS_PER_REQUEST = 1; + + /** Provider interface for creating CassandraRecordWriter instances. */ + @FunctionalInterface + interface RecordWriterProvider { + CassandraRecordWriter create( + ClusterBuilder clusterBuilder, CassandraSinkConfig config); + } + + private final Semaphore semaphore; + private final MailboxExecutor mailboxExecutor; + private final RequestConfiguration requestConfiguration; + private final Counter successfulRecordsCounter; + private final Counter failedRecordsCounter; + private final Counter retryCounter; + private final CassandraFailureHandler failureHandler; + private final CassandraRecordWriter cassandraRecordWriter; + private final Histogram writeLatencyHistogram; + + /** Creates a new CassandraSinkWriter. */ + public CassandraSinkWriter( + RequestConfiguration requestConfiguration, + CassandraSinkConfig cassandraSinkConfig, + ClusterBuilder clusterBuilder, + CassandraFailureHandler failureHandler, + MailboxExecutor mailboxExecutor, + SinkWriterMetricGroup metricGroup) { + + this( + requestConfiguration, + cassandraSinkConfig, + clusterBuilder, + failureHandler, + mailboxExecutor, + metricGroup, + RecordWriterFactory::create); + } + + /** Creates a new CassandraSinkWriter with injectable RecordWriterProvider for testing. */ + CassandraSinkWriter( + RequestConfiguration requestConfiguration, + CassandraSinkConfig cassandraSinkConfig, + ClusterBuilder clusterBuilder, + CassandraFailureHandler failureHandler, + MailboxExecutor mailboxExecutor, + SinkWriterMetricGroup metricGroup, + RecordWriterProvider recordWriterProvider) { + this.semaphore = new Semaphore(requestConfiguration.getMaxConcurrentRequests(), true); + this.requestConfiguration = requestConfiguration; + this.mailboxExecutor = mailboxExecutor; + this.failureHandler = failureHandler; + ClosureCleaner.clean(clusterBuilder, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true); + this.successfulRecordsCounter = metricGroup.getIOMetricGroup().getNumRecordsOutCounter(); + this.failedRecordsCounter = metricGroup.getNumRecordsOutErrorsCounter(); + this.retryCounter = metricGroup.counter("retries"); + this.writeLatencyHistogram = + metricGroup.histogram( + "writeLatencyMillis", new DescriptiveStatisticsHistogram(1000)); + metricGroup.gauge( + "inflightRequests", + () -> + requestConfiguration.getMaxConcurrentRequests() + - semaphore.availablePermits()); + this.cassandraRecordWriter = + recordWriterProvider.create(clusterBuilder, cassandraSinkConfig); + } + + /** + * Executes a runnable on the mailbox executor, ensuring permits are released if scheduling + * fails. + * + * @param action the task to execute + * @param description description of the task for debugging + */ + private void executeOnMailbox( + ThrowingRunnable action, String description) { + try { + mailboxExecutor.execute(action, description); + } catch (Throwable ex) { + semaphore.release(); + throw ex; + } + } + + /** + * Writes an input element to Cassandra asynchronously. + * + * @param input the input element + * @param context the sink context + * @throws IOException if an unrecoverable error occurs + * @throws InterruptedException if the thread is interrupted while waiting for a permit + */ + @Override + public void write(INPUT input, Context context) throws IOException, InterruptedException { + acquirePermitWithTimeout(); + submitWithRetry(input, 0); + } + + /** + * Acquires a permit with timeout. + * + * @throws IOException if timeout is exceeded + * @throws InterruptedException if the thread is interrupted while waiting + */ + private void acquirePermitWithTimeout() throws IOException, InterruptedException { + long startTime = System.nanoTime(); + long timeoutNanos = requestConfiguration.getMaxTimeout().toNanos(); + + while (!semaphore.tryAcquire(PERMITS_PER_REQUEST)) { + if (System.nanoTime() - startTime > timeoutNanos) { + throw new IOException( + String.format( + "Failed to acquire %d out of %d permits to send value within %s", + PERMITS_PER_REQUEST, + requestConfiguration.getMaxConcurrentRequests(), + requestConfiguration.getMaxTimeout())); + } + mailboxExecutor.yield(); + } + } + + /** + * Submits a record for writing with retry logic. + * + * @param element the element to write + * @param attempt current attempt number (0-based) + */ + void submitWithRetry(INPUT element, int attempt) { + try { + long startTimeNanos = System.nanoTime(); + ListenableFuture future = cassandraRecordWriter.write(element); + + Futures.addCallback( + future, + new FutureCallback() { + @Override + public void onSuccess(@Nullable ResultSet resultSet) { + long cassandraLatencyMillis = + TimeUnit.NANOSECONDS.toMillis( + System.nanoTime() - startTimeNanos); + executeOnMailbox( + () -> { + writeLatencyHistogram.update(cassandraLatencyMillis); + semaphore.release(); + successfulRecordsCounter.inc(); + }, + "cassandra-success"); + } + + @Override + public void onFailure(@Nonnull Throwable t) { + executeOnMailbox( + () -> handleFailure(element, t, attempt), "cassandra-failure"); + } + }, + MoreExecutors.directExecutor()); + + } catch (Exception e) { + executeOnMailbox(() -> handleFailure(element, e, attempt), "handle-sync-write-failure"); + } + } + + /** + * Handles write failures with retry logic and failure classification. + * + * @param element the element that failed to write + * @param cause the throwable that caused the failure + * @param attempt current attempt number (0-based) + * @throws IOException if the failure should fail the sink + */ + private void handleFailure(INPUT element, Throwable cause, int attempt) throws IOException { + final boolean isFatal = failureHandler.isFatal(cause); + final int maxRetries = requestConfiguration.getMaxRetries(); + final boolean hasRetriesLeft = !isFatal && attempt < maxRetries; + + if (hasRetriesLeft) { + retryCounter.inc(); + submitWithRetry(element, attempt + 1); // keep the permit + return; + } + + // Terminal path + failedRecordsCounter.inc(); + try { + if (!isFatal) { + // retries exhausted but not fatal => surface as MaxRetriesExceededException + failureHandler.onFailure( + new MaxRetriesExceededException(element, attempt + 1, cause)); + } else { + // fatal => immediate fail + failureHandler.onFailure(cause); + } + } finally { + semaphore.release(); + } + } + + /** + * Flushes all pending write operations by waiting for all in-flight requests to complete. + * + * @param endOfInput whether this is the final flush call + * @throws IOException if an error occurs during flushing + * @throws InterruptedException if the thread is interrupted while waiting + */ + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + try { + executePendingTasks(); + } catch (TimeoutException e) { + throw new IOException("Unable to flush within timeout: " + e.getMessage(), e); + } + } + + /** + * Waits for all pending tasks to complete within the configured timeout. + * + * @throws InterruptedException if the thread is interrupted while waiting + * @throws TimeoutException if the timeout is exceeded + */ + void executePendingTasks() throws InterruptedException, TimeoutException { + final int maxConcurrentRequests = requestConfiguration.getMaxConcurrentRequests(); + final long deadlineNanos = + System.nanoTime() + requestConfiguration.getFlushTimeout().toNanos(); + + if (semaphore.availablePermits() >= maxConcurrentRequests) { + return; + } + + while (System.nanoTime() < deadlineNanos) { + if (semaphore.availablePermits() >= maxConcurrentRequests) { + return; + } + mailboxExecutor.yield(); + } + + final int available = semaphore.availablePermits(); + final int pendingPermits = Math.max(0, maxConcurrentRequests - available); + + throw new TimeoutException( + String.format( + "Timeout after %s while waiting for %d in-flight Cassandra requests to complete. " + + "Only %d of %d permits available.", + requestConfiguration.getFlushTimeout(), + pendingPermits, + available, + maxConcurrentRequests)); + } + + /** + * Closes the sink writer, waiting for all pending operations to complete. + * + * @throws Exception if an error occurs during cleanup + */ + @Override + public void close() throws Exception { + try { + executePendingTasks(); + } finally { + cassandraRecordWriter.close(); + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/DynamicPlannerAssembler.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/DynamicPlannerAssembler.java new file mode 100644 index 0000000..276913c --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/DynamicPlannerAssembler.java @@ -0,0 +1,61 @@ +package org.apache.flink.connector.cassandra.sink.assembler; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.SinkPluggable; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; +import org.apache.flink.connector.cassandra.sink.planner.core.strategy.InsertStrategy; +import org.apache.flink.connector.cassandra.sink.planner.core.strategy.PlannerStrategy; +import org.apache.flink.connector.cassandra.sink.planner.core.strategy.UpdateStrategy; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.util.Preconditions; + +/** + * Assembler for dynamic (pluggable) sink configurations. + * + *

Creates a StatementPlanner using pluggable resolvers and customizers from the configuration. + * The planner strategy (INSERT vs UPDATE) is determined by the column value resolver's kind. + * + * @param the input record type + */ +public class DynamicPlannerAssembler implements PlannerAssembler { + + private final CqlSinkConfig config; + + public DynamicPlannerAssembler(CqlSinkConfig config) { + this.config = config; + } + + @Override + public StatementPlanner assemble() { + SinkPluggable pluggable = config.getPluggable(); + Preconditions.checkArgument( + pluggable != null, "SinkPluggable cannot be null in DYNAMIC mode"); + Preconditions.checkArgument( + pluggable.getTableResolver() != null, + "TableResolver cannot be null in DYNAMIC mode"); + Preconditions.checkArgument( + pluggable.getColumnValueResolver() != null, + "ColumnValueResolver cannot be null in DYNAMIC mode"); + + // Choose strategy based on resolver's kind + PlannerStrategy strategy; + ColumnValueResolver.Kind resolverKind = pluggable.getColumnValueResolver().kind(); + switch (resolverKind) { + case INSERT: + strategy = new InsertStrategy<>(); + break; + case UPDATE: + strategy = new UpdateStrategy<>(); + break; + default: + throw new IllegalArgumentException("Unknown resolver kind: " + resolverKind); + } + + return new StatementPlanner<>( + pluggable.getTableResolver(), + pluggable.getColumnValueResolver(), + strategy, + pluggable.getCqlClauseResolver(), + pluggable.getStatementCustomizer()); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/PlannerAssembler.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/PlannerAssembler.java new file mode 100644 index 0000000..bea9c88 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/PlannerAssembler.java @@ -0,0 +1,14 @@ +package org.apache.flink.connector.cassandra.sink.assembler; + +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; + +/** Interface for assembling {@link StatementPlanner} instances from sink configurations. */ +public interface PlannerAssembler { + + /** + * Assembles a StatementPlanner from the given configuration. + * + * @return the assembled statement planner + */ + StatementPlanner assemble(); +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/StaticPlannerAssembler.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/StaticPlannerAssembler.java new file mode 100644 index 0000000..925c796 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/assembler/StaticPlannerAssembler.java @@ -0,0 +1,123 @@ +package org.apache.flink.connector.cassandra.sink.assembler; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.NoOpCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.NullUnsettingCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.strategy.StaticInsertStrategy; +import org.apache.flink.connector.cassandra.sink.planner.core.strategy.StaticUpdateStrategy; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.FixedColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.FixedTableResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver; +import org.apache.flink.connector.cassandra.sink.util.QueryParser; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Collections; +import java.util.List; + +/** + * Assembler for static (CQL query-based) sink configurations. + * + *

Creates a StatementPlanner by parsing the provided CQL query to determine INSERT or UPDATE + * mode, then building appropriate resolvers and strategies. + * + * @param the input record type + */ +public class StaticPlannerAssembler implements PlannerAssembler { + + private final CqlSinkConfig config; + + public StaticPlannerAssembler(CqlSinkConfig config) { + this.config = config; + } + + @Override + public StatementPlanner assemble() { + String query = config.getQuery(); + Preconditions.checkArgument(!StringUtils.isEmpty(query), "Query cannot be empty."); + StaticQueryType staticQueryType = determineQueryType(query); + switch (staticQueryType) { + case INSERT: + return assembleInsertPlanner(query); + case UPDATE: + return assembleUpdatePlanner(query); + default: + throw new IllegalArgumentException( + String.format( + "Static mode only supports INSERT and UPDATE queries. Got: %s", + query)); + } + } + + /** Determines the type of CQL query (INSERT or UPDATE). */ + private StaticQueryType determineQueryType(String query) { + String normalizedQuery = query.toUpperCase().trim(); + if (normalizedQuery.startsWith("INSERT")) { + return StaticQueryType.INSERT; + } else if (normalizedQuery.startsWith("UPDATE")) { + return StaticQueryType.UPDATE; + } else { + return StaticQueryType.UNKNOWN; + } + } + + /** Assembles a StatementPlanner for INSERT queries. */ + private StatementPlanner assembleInsertPlanner(String query) { + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + TableResolver tableResolver = + createTableResolver(queryInfo.getKeyspace(), queryInfo.getTableName()); + ColumnValueResolver resolver = + createColumnValueResolver( + config, queryInfo.getColumnNames(), Collections.emptyList()); + StatementCustomizer customizer = createStatementCustomizer(); + + return new StatementPlanner<>( + tableResolver, resolver, new StaticInsertStrategy<>(), null, customizer); + } + + /** Assembles a StatementPlanner for UPDATE queries. */ + private StatementPlanner assembleUpdatePlanner(String query) { + QueryParser.UpdateQueryInfo updateInfo = QueryParser.parseUpdateQuery(query); + + TableResolver tableResolver = + createTableResolver(updateInfo.getKeyspace(), updateInfo.getTableName()); + ColumnValueResolver resolver = + createColumnValueResolver( + config, updateInfo.getSetColumns(), updateInfo.getWhereColumns()); + StatementCustomizer customizer = createStatementCustomizer(); + + return new StatementPlanner<>( + tableResolver, resolver, new StaticUpdateStrategy<>(), null, customizer); + } + + /** Creates a FixedTableResolver for the given keyspace and table. */ + private TableResolver createTableResolver(String keyspace, String tableName) { + return new FixedTableResolver<>(keyspace, tableName); + } + + /** Creates the appropriate StatementCustomizer based on config. */ + private StatementCustomizer createStatementCustomizer() { + return config.getIgnoreNullFields() + ? new NullUnsettingCustomizer<>() + : new NoOpCustomizer<>(); + } + + /** Enum for query types supported in static mode. */ + private enum StaticQueryType { + INSERT, + UPDATE, + UNKNOWN + } + + /** Creates a ColumnValueResolver based on RecordFormatType. */ + private static ColumnValueResolver createColumnValueResolver( + CqlSinkConfig config, List columnNames, List whereColumns) { + return new FixedColumnValueResolver<>( + config.getRecordFormatType(), columnNames, whereColumns); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CassandraPluggableConfig.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CassandraPluggableConfig.java new file mode 100644 index 0000000..4ebed57 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CassandraPluggableConfig.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.cassandra.sink.planner.SinkPluggable; + +import javax.annotation.Nullable; + +/** + * Common interface for Cassandra sink configurations that support pluggable components. + * + *

This interface extends the base {@link CassandraSinkConfig} to add support for customizable + * components: + * + *

    + *
  • {@link org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver} + *
  • {@link org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver} + *
  • {@link + * org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer} + *
  • {@link org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver} + *
+ * + *

All concrete CqlSink implementations would use this interface to provide a uniform way for + * users to plug in custom behavior. + * + * @param the input record type + */ +@PublicEvolving +public interface CassandraPluggableConfig extends CassandraSinkConfig { + + /** + * Gets the pluggable components configuration. + * + * @return the sink pluggable containing optional custom implementations, or null if not set + */ + @Nullable + SinkPluggable getPluggable(); + + /** + * Sets the pluggable components configuration. + * + * @param pluggable the sink pluggable to use + * @return this configuration for fluent chaining + */ + CassandraPluggableConfig withPluggable(SinkPluggable pluggable); +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CassandraSinkConfig.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CassandraSinkConfig.java new file mode 100644 index 0000000..58e508b --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CassandraSinkConfig.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * Base configuration interface for all Cassandra sink configurations. + * + *

The generic type parameter {@code } represents the type of records the sink will + * consume. Values include: + * + *

    + *
  • {@link org.apache.flink.types.Row} for DataStream API jobs using Rows + *
  • {@link org.apache.flink.api.java.tuple.Tuple} for Tuple types + *
  • {@link scala.Product} for Scala case classes + *
  • A user-defined POJO type for POJO sinks + *
+ * + * @param the input record type that will be written to Cassandra + */ +@PublicEvolving +public interface CassandraSinkConfig extends Serializable { + + /** + * Gets the record format type for this configuration. + * + * @return the format type used to determine the appropriate writer implementation + */ + RecordFormatType getRecordFormatType(); +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CqlSinkConfig.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CqlSinkConfig.java new file mode 100644 index 0000000..396f1aa --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/CqlSinkConfig.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.connector.cassandra.sink.planner.SinkPluggable; +import org.apache.flink.connector.cassandra.sink.planner.core.resolution.ResolutionMode; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.StringUtils; + +import javax.annotation.Nullable; + +/** + * Configuration class for CQL-based Cassandra sink configurations. + * + *

This holds common configuration shared across different structured data types. + * + *

Modes + * + *

    + *
  • STATIC: set via {@link #withQuery(String)}; user supplies exact CQL (INSERT/UPDATE). + *
  • DYNAMIC: set via {@link #withPluggable(SinkPluggable)}; tables/columns resolved per + * record by the pluggable resolvers. {@code query} must be unset. + *
+ * + * @param the input record type + */ +@PublicEvolving +public class CqlSinkConfig implements CassandraPluggableConfig { + + private static final long serialVersionUID = 1L; + + private Boolean ignoreNullFields; + private String query; + private final RecordFormatType recordFormatType; + private SinkPluggable pluggable; + + /** Default value for ignoreNullFields. */ + private static final Boolean DEFAULT_IGNORE_NULL_FIELDS = Boolean.FALSE; + + /** + * Creates a new CQL sink configuration with default settings. + * + * @param recordFormatType the format type for field extraction + */ + protected CqlSinkConfig(RecordFormatType recordFormatType) { + this.ignoreNullFields = DEFAULT_IGNORE_NULL_FIELDS; + this.query = null; + this.recordFormatType = recordFormatType; + } + + /** + * Gets the resolution mode based on the current configuration state. + * + * @return STATIC if query is set, DYNAMIC if pluggable is set, UNSET otherwise + */ + public ResolutionMode getResolutionMode() { + if (!StringUtils.isEmpty(query)) { + return ResolutionMode.STATIC; + } + if (pluggable != null) { + return ResolutionMode.DYNAMIC; + } + return ResolutionMode.UNSET; + } + + /** + * Gets whether null values should be unset during writes. + * + *

When true, bound null values are unset on the statement (INSERT or UPDATE), avoiding the + * creation of tombstones in Cassandra. + * + * @return true if null values should be unset + */ + public boolean getIgnoreNullFields() { + return ignoreNullFields; + } + + /** + * Gets the CQL query statement to execute. + * + * @return the CQL query statement (INSERT or UPDATE), or null when using dynamic/pluggable mode + */ + @Nullable + public String getQuery() { + return query; + } + + @Override + @Nullable + public SinkPluggable getPluggable() { + return pluggable; + } + + @Override + public RecordFormatType getRecordFormatType() { + return recordFormatType; + } + + @Override + public CqlSinkConfig withPluggable(SinkPluggable pluggable) { + Preconditions.checkState( + getResolutionMode() != ResolutionMode.STATIC, + "Cannot set pluggables when a query is already configured (STATIC and DYNAMIC are mutually exclusive)."); + Preconditions.checkArgument(pluggable != null, "pluggable cannot be null"); + this.pluggable = pluggable; + return this; + } + + /** + * Sets whether to unset null values during writes. + * + *

When true, bound null values are unset on the statement (INSERT or UPDATE), avoiding the + * creation of tombstones in Cassandra. + * + *

Note: This setting applies only in STATIC mode. In DYNAMIC mode (when using {@link + * #withPluggable(SinkPluggable)}), control null/unset behavior via a custom {@link + * org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer} in + * the pluggable configuration. + * + * @param ignoreNullFields true to unset null values + * @return this configuration for fluent chaining + * @throws IllegalStateException if called when DYNAMIC mode is already configured + */ + public CqlSinkConfig withIgnoreNullFields(boolean ignoreNullFields) { + Preconditions.checkState( + getResolutionMode() != ResolutionMode.DYNAMIC, + "ignoreNullFields is not supported in DYNAMIC mode; control null/unset behavior via StatementCustomizer in the SinkPluggable."); + this.ignoreNullFields = ignoreNullFields; + return this; + } + + /** + * Sets the CQL query for manual mode. + * + * @param query the CQL statement (INSERT or UPDATE) + * @return this configuration for fluent chaining + */ + public CqlSinkConfig withQuery(String query) { + Preconditions.checkState( + getResolutionMode() != ResolutionMode.DYNAMIC, + "Cannot set query when pluggables are already configured (STATIC and DYNAMIC are mutually exclusive)."); + Preconditions.checkArgument( + query != null && !StringUtils.isEmpty(query.trim()), "query must not be blank"); + this.query = query; + return this; + } + + /** + * Creates a CQL sink configuration for Row records. + * + *

Operates in Manual mode if {@code withQuery(...)} is called, or Dynamic mode if {@code + * withPluggable(...)} is called. Exactly one mode must be configured, otherwise the + * configuration will fail validation during sink creation. + * + * @return a new CQL sink configuration for Row records + */ + public static CqlSinkConfig forRow() { + return new CqlSinkConfig<>(RecordFormatType.ROW); + } + + /** + * Creates a CQL sink configuration for Tuple records. + * + *

Operates in Manual mode if {@code withQuery(...)} is called, or Dynamic mode if {@code + * withPluggable(...)} is called. Exactly one mode must be configured, otherwise the + * configuration will fail validation during sink creation. + * + * @return a new CQL sink configuration for Tuple records + */ + public static CqlSinkConfig forTuple() { + return new CqlSinkConfig<>(RecordFormatType.TUPLE); + } + + /** + * Creates a CQL sink configuration for Scala Product records. + * + *

Operates in Manual mode if {@code withQuery(...)} is called, or Dynamic mode if {@code + * withPluggable(...)} is called. Exactly one mode must be configured, otherwise the + * configuration will fail validation during sink creation. + * + * @return a new CQL sink configuration for Scala Product records + */ + public static CqlSinkConfig forScalaProduct() { + return new CqlSinkConfig<>(RecordFormatType.SCALA_PRODUCT); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/PojoSinkConfig.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/PojoSinkConfig.java new file mode 100644 index 0000000..981e751 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/PojoSinkConfig.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.connectors.cassandra.MapperOptions; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.StringUtils; + +import javax.annotation.Nullable; + +/** + * Configuration for writing POJO-based records to Cassandra using the DataStax Mapper. + * + *

This configuration is used with annotated Java classes that map directly to Cassandra tables + * using DataStax driver annotations like {@code @Table} and {@code @Column}. The POJO class should + * be properly annotated to define the table structure and field mappings. + * + *

Example POJO: + * + *

{@code
+ * @Table(keyspace = "example", name = "users")
+ * public class User {
+ *     @PartitionKey
+ *     private String id;
+ *
+ *     @Column(name = "full_name")
+ *     private String fullName;
+ *
+ *     // getters and setters
+ * }
+ * }
+ * + * @param the POJO type + */ +@PublicEvolving +public class PojoSinkConfig implements CassandraSinkConfig { + + private static final long serialVersionUID = 1L; + + private final Class pojoClass; + private final String keyspace; + @Nullable private final MapperOptions mapperOptions; + + /** + * Creates a new PojoSinkConfig. + * + * @param pojoClass the POJO class used for mapping. + * @param keyspace the Cassandra keyspace where the target table resides + * @param mapperOptions optional DataStax Mapper options (TTL, consistency level, etc.) + */ + public PojoSinkConfig( + Class pojoClass, String keyspace, @Nullable MapperOptions mapperOptions) { + Preconditions.checkArgument(pojoClass != null, "pojoClass must not be null."); + Preconditions.checkArgument(!StringUtils.isEmpty(keyspace), "Keyspace cannot be empty."); + this.pojoClass = pojoClass; + this.keyspace = keyspace; + this.mapperOptions = mapperOptions; + } + + /** + * Gets the POJO class used for mapping. + * + * @return the input class type used to configure the DataStax Mapper + */ + public Class getPojoClass() { + return pojoClass; + } + + /** + * Gets the keyspace where the target Cassandra table resides. + * + * @return the keyspace name + */ + public String getKeyspace() { + return keyspace; + } + + /** + * Gets the optional {@link MapperOptions} for configuring the DataStax Mapper. + * + * @return MapperOptions such as TTL, consistency level, etc., or null if not set + */ + @Nullable + public MapperOptions getMapperOptions() { + return mapperOptions; + } + + @Override + public RecordFormatType getRecordFormatType() { + return RecordFormatType.POJO; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/RecordFormatType.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/RecordFormatType.java new file mode 100644 index 0000000..e79f4cb --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/RecordFormatType.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config; + +import org.apache.flink.annotation.PublicEvolving; + +/** Enum of the supported format types. */ +@PublicEvolving +public enum RecordFormatType { + + /** Flink Row type. Fields are extracted using {@code row.getField(index)}. */ + ROW("row"), + + /** + * Flink Tuple types (Tuple1, Tuple2, etc.). Fields are extracted using {@code + * tuple.getField(index)}. + */ + TUPLE("tuple"), + + /** + * Scala Product types (case classes). Fields are extracted using {@code + * product.productElement(index)}. + */ + SCALA_PRODUCT("scala_product"), + + /** + * POJO types with DataStax Mapper annotations. Serialization is handled by the DataStax Mapper. + */ + POJO("pojo"); + + private final String name; + + RecordFormatType(String name) { + this.name = name; + } + + /** + * Gets the string representation of this record format type. + * + * @return the format type name + */ + public String getName() { + return name; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/write/RequestConfiguration.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/write/RequestConfiguration.java new file mode 100644 index 0000000..c46d5f3 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/config/write/RequestConfiguration.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config.write; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.time.Duration; + +/** + * Configuration for controlling Cassandra sink request behavior including retry handling, + * connection concurrency, and timeout limits. + * + *

This configuration determines how the sink handles: + * + *

    + *
  • Maximum number of concurrent requests to Cassandra + *
  • Number of retry attempts for failed requests + *
  • Timeout for acquiring permits during normal writes + *
  • Timeout for draining all in-flight requests during flush/checkpoint + *
+ * + *

Important: Separate timeouts for different operations are used. + * + *

    + *
  • {@code maxTimeout} - Used during regular write operations when acquiring permits. Should be + * generous to handle normal backpressure without failing the job. + *
  • {@code flushTimeout} - Used during flush/checkpoint when draining all in-flight requests. + * Should be bounded to ensure checkpoints complete or fail promptly. + *
+ */ +@PublicEvolving +public class RequestConfiguration implements Serializable { + + private static final long serialVersionUID = 1L; + + /** Default maximum number of concurrent Cassandra requests. */ + public static final int DEFAULT_MAX_CONCURRENT_REQUESTS = Integer.MAX_VALUE; + + /** Default number of retries on failure. */ + public static final int DEFAULT_MAX_RETRIES = 0; + + /** Default maximum timeout for acquiring permits during write operations. */ + public static final Duration DEFAULT_MAX_TIMEOUT = Duration.ofMinutes(1); + + /** Default timeout for draining all requests during flush/checkpoint. */ + public static final Duration DEFAULT_FLUSH_TIMEOUT = Duration.ofSeconds(30); + + private final int maxRetries; + private final int maxConcurrentRequests; + private final Duration maxTimeout; + private final Duration flushTimeout; + + /** + * Private constructor. Use {@link Builder} to create instances. + * + * @param maxRetries number of retry attempts per record + * @param maxConcurrentRequests maximum number of concurrent Cassandra requests + * @param maxTimeout maximum time to wait for acquiring permits during writes + * @param flushTimeout maximum time to wait for draining requests during flush + */ + private RequestConfiguration( + int maxRetries, int maxConcurrentRequests, Duration maxTimeout, Duration flushTimeout) { + this.maxRetries = maxRetries; + this.maxConcurrentRequests = maxConcurrentRequests; + this.maxTimeout = maxTimeout; + this.flushTimeout = flushTimeout; + } + + /** + * Gets the maximum number of retry attempts for failed requests. + * + * @return maximum retry attempts + */ + public int getMaxRetries() { + return maxRetries; + } + + /** + * Gets the maximum number of concurrent requests allowed to Cassandra. + * + * @return maximum concurrent requests + */ + public int getMaxConcurrentRequests() { + return maxConcurrentRequests; + } + + /** + * Gets the maximum timeout for acquiring permits during write operations. + * + *

This timeout is used when acquiring permits for individual writes. It should be generous + * enough to handle normal backpressure without causing unnecessary job failures. + * + * @return maximum timeout duration for permit acquisition + */ + public Duration getMaxTimeout() { + return maxTimeout; + } + + /** + * Gets the timeout for draining all in-flight requests during flush operations. + * + *

This timeout is used during checkpoints when waiting for all pending requests to complete. + * It should be bounded to ensure checkpoints either succeed or fail within a reasonable time, + * avoiding checkpoint timeout issues. + * + * @return flush timeout duration + */ + public Duration getFlushTimeout() { + return flushTimeout; + } + + /** + * Creates a new builder for RequestConfiguration. + * + * @return a new builder instance + */ + public static Builder builder() { + return new Builder(); + } + + /** Builder for {@link RequestConfiguration}. */ + @PublicEvolving + public static class Builder { + + private Integer maxRetries = DEFAULT_MAX_RETRIES; + private Integer maxConcurrentRequests = DEFAULT_MAX_CONCURRENT_REQUESTS; + private Duration maxTimeout = DEFAULT_MAX_TIMEOUT; + private Duration flushTimeout = DEFAULT_FLUSH_TIMEOUT; + + /** + * Sets the maximum number of retries for failed requests. + * + * @param maxRetries maximum retry attempts (must be >= 0) + * @return this builder + * @throws IllegalArgumentException if maxRetries is negative + */ + public Builder setMaxRetries(Integer maxRetries) { + Preconditions.checkArgument( + maxRetries != null && maxRetries >= 0, "maxRetries must be non-null and >= 0"); + this.maxRetries = maxRetries; + return this; + } + + /** + * Sets the maximum number of concurrent Cassandra requests. + * + * @param maxConcurrentRequests maximum concurrent requests (must be > 0) + * @return this builder + * @throws IllegalArgumentException if maxConcurrentRequests is <= 0 + */ + public Builder setMaxConcurrentRequests(Integer maxConcurrentRequests) { + Preconditions.checkArgument( + maxConcurrentRequests != null && maxConcurrentRequests > 0, + "maxConcurrentRequests must be non-null and > 0"); + this.maxConcurrentRequests = maxConcurrentRequests; + return this; + } + + /** + * Sets the maximum timeout duration for acquiring permits during write operations. + * + * @param maxTimeout timeout duration (must be positive) + * @return this builder + * @throws IllegalArgumentException if maxTimeout is not positive + */ + public Builder setMaxTimeout(Duration maxTimeout) { + Preconditions.checkArgument( + maxTimeout != null && !maxTimeout.isZero() && !maxTimeout.isNegative(), + "maxTimeout must be non-null and positive"); + this.maxTimeout = maxTimeout; + return this; + } + + /** + * Sets the timeout for draining all in-flight requests during flush operations. + * + *

This timeout should be aligned with your checkpoint interval and timeout settings. A + * good rule of thumb is to set it to less than half of your checkpoint timeout to allow + * time for other checkpoint operations. + * + * @param flushTimeout flush timeout duration (must be positive) + * @return this builder + * @throws IllegalArgumentException if flushTimeout is not positive + */ + public Builder setFlushTimeout(Duration flushTimeout) { + Preconditions.checkArgument( + flushTimeout != null && !flushTimeout.isZero() && !flushTimeout.isNegative(), + "flushTimeout must be non-null and positive"); + this.flushTimeout = flushTimeout; + return this; + } + + /** + * Builds the {@link RequestConfiguration} using the configured values or defaults where + * values were not set. + * + * @return configured RequestConfiguration instance + */ + public RequestConfiguration build() { + return new RequestConfiguration( + maxRetries, maxConcurrentRequests, maxTimeout, flushTimeout); + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/CassandraFailureHandler.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/CassandraFailureHandler.java new file mode 100644 index 0000000..bb47899 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/CassandraFailureHandler.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.exception; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Optional; + +/** + * Handler for Cassandra write failures in the sink writer. + * + *

This handler provides policies for how to respond to failures during Cassandra writes, + * including fatal exception classification and failure reporting. + * + *

The handler determines whether exceptions should be treated as fatal (failing the sink) or + * retriable (subject to retry logic based on the configured retry policy). + * + *

Important: When the sink exhausts its retry budget for a record, it calls + * {@link #onFailure(Throwable)} with a {@link MaxRetriesExceededException}. The default handler + * fails the job to preserve at-least-once semantics. Providing a custom classifier does not change + * this; to opt into best-effort delivery, subclass the handler and override {@code onFailure}. + */ +public class CassandraFailureHandler implements Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(CassandraFailureHandler.class); + + private static final long serialVersionUID = 1L; + + /** + * Determines whether an exception should be considered fatal and fail the sink. + * + * @param cause the exception to classify + * @return true if the exception is fatal and should fail the sink + */ + public boolean isFatal(Throwable cause) { + return CassandraFatalExceptionClassifier.getFatalExceptionMessage(cause).isPresent(); + } + + /** + * Central failure policy that handles terminal failures. + * + *

This method is called when a write operation fails terminally (either immediately fatal or + * after exhausting retries). The default implementation: + * + *

    + *
  • Always fails on {@link MaxRetriesExceededException} to preserve at-least-once semantics + *
  • Fails on exceptions classified as fatal by the classifier + *
  • Logs and swallows other non-fatal exceptions (best-effort for transient errors) + *
+ * + *

Users who want best-effort delivery (allowing record loss) can override this method to + * swallow {@link MaxRetriesExceededException}. + * + * @param cause the exception that caused the failure + * @throws IOException if the failure should fail the sink + */ + public void onFailure(Throwable cause) throws IOException { + if (cause instanceof MaxRetriesExceededException) { + throw new IOException( + "Cassandra write failed after exhausting retries. " + + "To allow record loss, override CassandraFailureHandler.onFailure()", + cause); + } + Optional fatalMessage = + CassandraFatalExceptionClassifier.getFatalExceptionMessage(cause); + if (fatalMessage.isPresent()) { + throw new IOException(fatalMessage.get(), cause); + } + LOG.warn( + "Ignoring non-fatal Cassandra error (best-effort delivery): {}", + cause.getMessage(), + cause); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/CassandraFatalExceptionClassifier.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/CassandraFatalExceptionClassifier.java new file mode 100644 index 0000000..89c526a --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/CassandraFatalExceptionClassifier.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.exception; + +import com.datastax.driver.core.exceptions.AuthenticationException; +import com.datastax.driver.core.exceptions.CodecNotFoundException; +import com.datastax.driver.core.exceptions.DriverInternalError; +import com.datastax.driver.core.exceptions.FunctionExecutionException; +import com.datastax.driver.core.exceptions.InvalidQueryException; +import com.datastax.driver.core.exceptions.InvalidTypeException; +import com.datastax.driver.core.exceptions.SyntaxError; +import com.datastax.driver.core.exceptions.UnauthorizedException; +import com.datastax.driver.core.exceptions.UnsupportedFeatureException; +import com.datastax.driver.core.exceptions.UnsupportedProtocolVersionException; + +import javax.naming.ConfigurationException; + +import java.util.Optional; + +/** + * Defines a central list of fatal Cassandra exceptions that should not be retried. + * + *

This classifier helps distinguish between retriable transient errors (like timeouts or + * temporary unavailability) and fatal errors that indicate configuration problems or permanent + * failures that will not be resolved by retrying. + * + *

Fatal exceptions include: + * + *

    + *
  • Authentication and authorization failures + *
  • Query syntax and schema errors + *
  • Driver configuration and codec issues + *
  • Protocol version mismatches + *
+ */ +public class CassandraFatalExceptionClassifier { + + /** Enumeration of known fatal exception types with descriptive error messages. */ + private enum FatalExceptionType { + INVALID_QUERY( + InvalidQueryException.class, "Encountered a non-recoverable InvalidQueryException"), + UNAUTHORIZED( + UnauthorizedException.class, "Encountered a non-recoverable UnauthorizedException"), + SYNTAX_ERROR(SyntaxError.class, "Encountered a non-recoverable SyntaxError"), + CONFIGURATION( + ConfigurationException.class, + "Encountered a non-recoverable ConfigurationException"), + AUTHENTICATION( + AuthenticationException.class, + "Encountered a non-recoverable AuthenticationException"), + INVALID_TYPE( + InvalidTypeException.class, "Encountered a non-recoverable InvalidTypeException"), + UNSUPPORTED_FEATURE( + UnsupportedFeatureException.class, + "Encountered a non-recoverable UnsupportedFeatureException"), + UNSUPPORTED_PROTOCOL( + UnsupportedProtocolVersionException.class, + "Encountered a non-recoverable UnsupportedProtocolVersionException"), + DRIVER_ERROR( + DriverInternalError.class, "Encountered a non-recoverable DriverInternalError"), + CODEC_NOT_FOUND( + CodecNotFoundException.class, + "Encountered a non-recoverable CodecNotFoundException"), + FUNCTION_EXECUTION( + FunctionExecutionException.class, + "Encountered a non-recoverable FunctionExecutionException"); + + private final Class exceptionClass; + private final String message; + + FatalExceptionType(Class exceptionClass, String message) { + this.exceptionClass = exceptionClass; + this.message = message; + } + } + + /** + * Checks if the given exception or any of its root causes is a fatal Cassandra exception and + * returns the specific error message if it is fatal. + * + * @param throwable the exception to check + * @return Optional containing the fatal error message, or empty if not fatal + */ + public static Optional getFatalExceptionMessage(Throwable throwable) { + if (throwable == null) { + return Optional.empty(); + } + + // Check the entire cause chain + Throwable current = throwable; + while (current != null) { + // Check against all fatal exception types + for (FatalExceptionType type : FatalExceptionType.values()) { + if (type.exceptionClass.isInstance(current)) { + return Optional.of(type.message); + } + } + current = current.getCause(); + } + return Optional.empty(); + } + + /** + * Checks if the given exception or any of its root causes is a fatal Cassandra exception. + * + * @param throwable the exception to check + * @return true if the exception is fatal and should not be retried + */ + public static boolean isFatalException(Throwable throwable) { + return getFatalExceptionMessage(throwable).isPresent(); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/MaxRetriesExceededException.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/MaxRetriesExceededException.java new file mode 100644 index 0000000..900b9de --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/exception/MaxRetriesExceededException.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.exception; + +import org.apache.flink.annotation.Internal; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** + * Exception thrown when maximum retry attempts are exhausted for a record. This is considered a + * terminal failure that by default fails the job to preserve at-least-once semantics. + */ +@Internal +public final class MaxRetriesExceededException extends IOException { + private static final long serialVersionUID = 1L; + private final int attempts; + private final String elementPreview; + + public MaxRetriesExceededException(@Nullable Object element, int attempts, Throwable cause) { + super( + "Max retries exceeded (attempts=" + + attempts + + ") for element=" + + summarize(element), + cause); + this.attempts = attempts; + this.elementPreview = summarize(element); + } + + public int getAttempts() { + return attempts; + } + + public String getElementPreview() { + return elementPreview; + } + + private static String summarize(@Nullable Object element) { + if (element == null) { + return "null"; + } + String str = String.valueOf(element); + return str.length() > 512 ? str.substring(0, 512) + "..." : str; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/SinkPluggable.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/SinkPluggable.java new file mode 100644 index 0000000..ec80339 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/SinkPluggable.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +/** + * Holder for all pluggable components that can be customized in the Cassandra sink. + * + *

All fields are nullable - when null, defaults will be used. + * + * @param the input record type + */ +@PublicEvolving +public final class SinkPluggable implements Serializable { + + private static final long serialVersionUID = 1L; + + private final TableResolver tableResolver; + private final ColumnValueResolver columnValueResolver; + @Nullable private final CqlClauseResolver cqlClauseResolver; + @Nullable private final StatementCustomizer statementCustomizer; + + SinkPluggable( + TableResolver tableResolver, + ColumnValueResolver columnValueResolver, + @Nullable CqlClauseResolver cqlClauseResolver, + @Nullable StatementCustomizer statementCustomizer) { + this.tableResolver = tableResolver; + this.columnValueResolver = columnValueResolver; + this.cqlClauseResolver = cqlClauseResolver; + this.statementCustomizer = statementCustomizer; + } + + /** + * Creates a new builder for SinkPluggable. + * + * @param the input record type + * @return a new builder instance + */ + public static Builder builder() { + return new Builder<>(); + } + + /** + * Gets the table resolver for dynamic table selection. + * + * @return the table resolver. + */ + public TableResolver getTableResolver() { + return tableResolver; + } + + /** + * Gets the column value resolver for dynamic column/value extraction. + * + * @return the column value resolver. + */ + public ColumnValueResolver getColumnValueResolver() { + return columnValueResolver; + } + + /** + * Gets the CQL clause resolver for dynamic CQL clause application. + * + * @return the CQL clause resolver, or null if not set + */ + @Nullable + public CqlClauseResolver getCqlClauseResolver() { + return cqlClauseResolver; + } + + /** + * Gets the statement customizer for per-statement modifications. + * + * @return the statement customizer, or null if not set + */ + @Nullable + public StatementCustomizer getStatementCustomizer() { + return statementCustomizer; + } + + /** + * Builder for {@link SinkPluggable}. + * + * @param the input record type + */ + @PublicEvolving + public static class Builder { + + private TableResolver tableResolver; + private ColumnValueResolver columnValueResolver; + @Nullable private CqlClauseResolver cqlClauseResolver; + @Nullable private StatementCustomizer statementCustomizer; + /** + * Sets the table resolver for dynamic table selection. + * + * @param tableResolver the table resolver to use + * @return this builder + */ + public Builder withTableResolver(TableResolver tableResolver) { + this.tableResolver = tableResolver; + return this; + } + + /** + * Sets the column value resolver for INSERT operations. + * + * @param columnValueResolver the resolver to use for INSERT operations + * @return this builder + */ + public Builder withColumnValueResolver( + ColumnValueResolver columnValueResolver) { + this.columnValueResolver = columnValueResolver; + return this; + } + + /** + * Sets the CQL clause resolver for dynamic CQL clause application. + * + * @param cqlClauseResolver the CQL clause resolver to use + * @return this builder + */ + public Builder withCqlClauseResolver( + @Nullable CqlClauseResolver cqlClauseResolver) { + this.cqlClauseResolver = cqlClauseResolver; + return this; + } + + /** + * Sets the statement customizer for additional statement configuration. + * + * @param statementCustomizer the statement customizer to use + * @return this builder + */ + public Builder withStatementCustomizer( + @Nullable StatementCustomizer statementCustomizer) { + this.statementCustomizer = statementCustomizer; + return this; + } + + /** + * Builds the SinkPluggable instance. + * + * @return a new SinkPluggable with the configured components + */ + public SinkPluggable build() { + Preconditions.checkArgument(tableResolver != null, "TableResolver cannot be null."); + Preconditions.checkArgument( + columnValueResolver != null, "ColumnValueResolver cannot be null."); + return new SinkPluggable<>( + tableResolver, columnValueResolver, cqlClauseResolver, statementCustomizer); + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/cache/PreparedStatementCache.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/cache/PreparedStatementCache.java new file mode 100644 index 0000000..290998e --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/cache/PreparedStatementCache.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.cache; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; + +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Session; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheStats; +import org.apache.commons.lang3.StringUtils; + +import java.util.concurrent.ExecutionException; + +/** + * Cache for prepared statements to avoid re-preparing identical queries using Guava Cache. + * + *

This is especially important for dynamic use cases where the same queries are used repeatedly + * across many records. Cache keys are the actual query strings to ensure different queries (e.g., + * with different TTL values) get separate cache entries. Uses Guava's thread-safe cache with + * configurable size limits and statistics tracking. + * + *

Cache Key Examples by Strategy: + * + *

    + *
  • StaticInsertStrategy: User-provided query string
    + * e.g., "INSERT INTO keyspace.events (id, timestamp, data) VALUES (?, ?, ?) USING TTL 3600" + *
  • StaticUpdateStrategy: User-provided query string
    + * e.g., "UPDATE keyspace.events SET data = ?, status = ? WHERE id = ? AND timestamp = ?" + *
  • InsertStrategy: Generated INSERT query (varies if dynamic table/columns)
    + * e.g., "INSERT INTO analytics.user_events (user_id,event_type,timestamp) VALUES (?,?,?)" + *
  • UpdateStrategy: Generated UPDATE query (varies if dynamic table/columns)
    + * e.g., "UPDATE analytics.user_profiles SET name=?,email=? WHERE user_id=?" + *
+ * + *

Important: One instance per Cassandra Session. Prepared statements are tied to a + * specific session and are not portable between sessions. + */ +@Internal +public class PreparedStatementCache implements AutoCloseable { + + private static final int DEFAULT_MAXIMUM_SIZE = 1000; + private final Cache cache; + + /** Creates a cache with default configuration. */ + public PreparedStatementCache() { + this(DEFAULT_MAXIMUM_SIZE); + } + + /** + * Creates a cache with specified maximum size. + * + *

Note: Prepared statements are not portable across sessions. Each session requires its own + * cache instance. + * + * @param maximumSize maximum number of entries to cache + */ + PreparedStatementCache(int maximumSize) { + this.cache = CacheBuilder.newBuilder().maximumSize(maximumSize).recordStats().build(); + } + + /** + * Gets a prepared statement from cache or creates a new one. + * + * @param session the Cassandra session + * @param cqlQuery Key for the cache + * @return the prepared statement + */ + public PreparedStatement getOrPrepare(Session session, String cqlQuery) { + Preconditions.checkArgument( + !StringUtils.isEmpty(cqlQuery), + "cqlQuery cannot be empty, it is the key for the cache."); + Preconditions.checkArgument(session != null, "session cannot be null."); + PreparedStatement ps = cache.getIfPresent(cqlQuery); + if (ps != null) { + return ps; + } + try { + return cache.get(cqlQuery, () -> session.prepare(cqlQuery)); + } catch (ExecutionException e) { + throw new RuntimeException( + String.format("Failed to prepare CQL [%s]", cqlQuery), e.getCause()); + } + } + + /** Returns the number of cached prepared statements. */ + public long size() { + return cache.size(); + } + + /** + * Returns cache statistics including hits, misses, evictions. + * + * @return cache statistics for monitoring + */ + public CacheStats getStats() { + return cache.stats(); + } + + /** + * Closes the cache and releases all resources. + * + *

This method clears all cached prepared statements and should be called when the cache is + * no longer needed to ensure proper cleanup. + */ + @Override + public void close() { + cache.invalidateAll(); + cache.cleanUp(); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/components/StatementPlanner.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/components/StatementPlanner.java new file mode 100644 index 0000000..22cced8 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/components/StatementPlanner.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.components; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.cache.PreparedStatementCache; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.NoOpCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.NoOpClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.core.strategy.PlannerStrategy; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver; +import org.apache.flink.connector.cassandra.sink.util.CqlStatementHelper; +import org.apache.flink.util.Preconditions; + +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; + +import javax.annotation.Nullable; + +/** + * Orchestrates the statement planning process by coordinating table resolution, column/value + * resolution, prepared statement caching, and value binding. + * + *

This is the core component that implements the planning algorithm: + * + *

    + *
  1. Resolve target table for the record + *
  2. Resolve columns and extract values for the record + *
  3. Get or create prepared statement for table+columns combination + *
  4. Bind values to prepared statement + *
+ * + * @param the input record type + */ +public class StatementPlanner implements AutoCloseable { + + private final TableResolver tableResolver; + private final ColumnValueResolver columnValueResolver; + private final PlannerStrategy strategy; + private final PreparedStatementCache preparedStatementCache; + private final CqlClauseResolver cqlClauseResolver; + private final StatementCustomizer statementCustomizer; + + /** + * Creates a statement planner with the given components. + * + * @param tableResolver resolves target table for records + * @param columnValueResolver resolves columns and values for records + * @param strategy handles mode-specific query generation and caching + * @param cqlClauseResolver resolves CQL modifiers (TTL, timestamp, IF conditions) - can be null + * @param statementCustomizer optional statement customizer - can be null + */ + public StatementPlanner( + TableResolver tableResolver, + ColumnValueResolver columnValueResolver, + PlannerStrategy strategy, + @Nullable CqlClauseResolver cqlClauseResolver, + @Nullable StatementCustomizer statementCustomizer) { + Preconditions.checkArgument(tableResolver != null, "tableResolver cannot be null"); + Preconditions.checkArgument( + columnValueResolver != null, "columnValueResolver cannot be null"); + Preconditions.checkNotNull(strategy != null, "strategy cannot be null"); + this.strategy = strategy; + this.tableResolver = tableResolver; + this.columnValueResolver = columnValueResolver; + this.preparedStatementCache = new PreparedStatementCache(); + this.cqlClauseResolver = + cqlClauseResolver == null ? new NoOpClauseResolver<>() : cqlClauseResolver; + this.statementCustomizer = + statementCustomizer == null ? new NoOpCustomizer<>() : statementCustomizer; + } + + /** + * Plans and creates a complete Cassandra Statement for a given input record. + * + *

This method orchestrates the complete process of converting an input record into a + * ready-to-execute Cassandra Statement by performing the following steps in order: + * + *

    + *
  1. Table Resolution: Determines target keyspace.table for the record. Resolve + * target table (keyspace.tablename) for this record + *
  2. Column/Value Extraction: Resolves which columns to write and their values + *
  3. Query Generation & Caching: Generates CQL query (INSERT/UPDATE) with modifiers + * and caches prepared statement + *
  4. Value Binding: Binds extracted values to prepared statement parameters + *
  5. Statement Customization: Applies statement-level settings like consistency, + * timeouts (if customizer set) + *
+ */ + public Statement plan(INPUT record, Session session, CqlSinkConfig config) { + TableRef targetTable = tableResolver.resolve(record); + ResolvedWrite resolvedWrite = columnValueResolver.resolve(record); + PlannerStrategy.QueryWithBindings qb = + strategy.getQueryWithBindings( + targetTable, config, cqlClauseResolver, resolvedWrite, record); + PreparedStatement preparedStatement = + preparedStatementCache.getOrPrepare(session, qb.query); + Object[] bindingValues = strategy.getOrderedBindValues(resolvedWrite, qb.clauseBindings); + Statement boundStatement = CqlStatementHelper.bind(preparedStatement, bindingValues); + statementCustomizer.apply(boundStatement, record); + return boundStatement; + } + + /** Clears the prepared statement cache to release resources. */ + public void close() { + if (preparedStatementCache != null) { + preparedStatementCache.close(); + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/components/StatementPlannerFactory.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/components/StatementPlannerFactory.java new file mode 100644 index 0000000..d9b0416 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/components/StatementPlannerFactory.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.components; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.cassandra.sink.assembler.DynamicPlannerAssembler; +import org.apache.flink.connector.cassandra.sink.assembler.StaticPlannerAssembler; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.resolution.ResolutionMode; + +/** + * Factory for creating StatementPlanner's with appropriate strategies based on sink configuration. + */ +@Internal +public final class StatementPlannerFactory { + + /** + * Creates a StatementPlanner for the given configuration. + * + * @param config the sink configuration + * @param the record type + * @return a configured StatementPlanner + */ + public static StatementPlanner create(CqlSinkConfig config) { + ResolutionMode mode = config.getResolutionMode(); + switch (mode) { + case UNSET: + throw new IllegalArgumentException( + String.format( + "Invalid ResolutionMode %s, Planner cannot be constructed.", mode)); + case DYNAMIC: + return new DynamicPlannerAssembler<>(config).assemble(); + case STATIC: + return new StaticPlannerAssembler<>(config).assemble(); + } + return null; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NoOpCustomizer.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NoOpCustomizer.java new file mode 100644 index 0000000..037dcc5 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NoOpCustomizer.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.customization; + +import org.apache.flink.annotation.Internal; + +import com.datastax.driver.core.Statement; + +/** + * A no-operation statement customizer that performs no modifications. + * + *

This is used as a default when no customization is needed. + */ +@Internal +public class NoOpCustomizer implements StatementCustomizer { + + @Override + public void apply(Statement statement, INPUT input) {} +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NullUnsettingCustomizer.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NullUnsettingCustomizer.java new file mode 100644 index 0000000..bd60e6a --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NullUnsettingCustomizer.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.customization; + +import org.apache.flink.annotation.Internal; + +import com.datastax.driver.core.BoundStatement; +import com.datastax.driver.core.Statement; + +/** + * Internal customizer that unsets null values in Cassandra bound statements. + * + *

This customizer implements the null-ignoring behavior when {@code ignoreNullFields=true} in + * Static {@link org.apache.flink.connector.cassandra.sink.planner.core.resolution.ResolutionMode}. + * When a bound statement has null values, they are unset, effectively excluding those columns from + * the INSERT/UPDATE operation. + * + *

Example 1: INSERT operation + * + *

{@code
+ * // Given a Cassandra table:
+ * // CREATE TABLE users (id int PRIMARY KEY, name text, email text, phone text);
+ *
+ * // Case 1: INSERT with null value (without NullUnsettingCustomizer)
+ * // Record: User(id=2, name='Bob', email=null, phone='+1234567890')
+ * // Generated statement: INSERT INTO users (id, name, email, phone) VALUES (?, ?, ?, ?)
+ * // Bound values: [2, 'Bob', null, '+1234567890']
+ * // Result in Cassandra: id=2, name='Bob', email=null, phone='+1234567890'
+ *
+ * // Case 2: INSERT with unset (with NullUnsettingCustomizer)
+ * // Same Record: User(id=2, name='Bob', email=null, phone='+1234567890')
+ * // Generated statement: INSERT INTO users (id, name, email, phone) VALUES (?, ?, ?, ?)
+ * // After customizer: INSERT INTO users (id, name, phone) VALUES (?, ?, ?)
+ * // Bound values: [2, 'Bob', '+1234567890']
+ * // Result in Cassandra: id=2, name='Bob', phone='+1234567890'
+ * // The email column is simply not set
+ * }
+ * + *

Example 2: UPDATE operation + * + *

{@code
+ * // Given an existing row in Cassandra:
+ * // id=1, name='Alice', email='alice@example.com', phone='+1234567890'
+ *
+ * // Case 1: UPDATE with null value (without NullUnsettingCustomizer)
+ * // Record: User(id=1, name='Alice Updated', email=null, phone='+9876543210')
+ * // Generated statement: UPDATE users SET name=?, email=?, phone=? WHERE id=?
+ * // Bound values: ['Alice Updated', null, '+9876543210', 1]
+ * // Result: id=1, name='Alice Updated', email=null (deleted), phone='+9876543210'
+ *
+ * // Case 2: UPDATE with unset (with NullUnsettingCustomizer)
+ * // Same Record: User(id=1, name='Alice Updated', email=null, phone='+9876543210')
+ * // Generated statement: UPDATE users SET name=?, email=?, phone=? WHERE id=?
+ * // After customizer: UPDATE users SET name=?, phone=? WHERE id=?
+ * // Bound values: ['Alice Updated', '+9876543210', 1]
+ * // Result: id=1, name='Alice Updated', email='alice@example.com', phone='+9876543210'
+ * // The email column keeps its existing value ('alice@example.com')
+ * }
+ * + *

Important: This customizer only works with {@link BoundStatement}s. + */ +@Internal +public class NullUnsettingCustomizer implements StatementCustomizer { + + @Override + public void apply(Statement statement, INPUT input) { + if (!(statement instanceof BoundStatement)) { + return; + } + + BoundStatement boundStatement = (BoundStatement) statement; + int variableCount = boundStatement.preparedStatement().getVariables().size(); + + for (int i = 0; i < variableCount; i++) { + if (boundStatement.isSet(i) && boundStatement.isNull(i)) { + boundStatement.unset(i); + } + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/StatementCustomizer.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/StatementCustomizer.java new file mode 100644 index 0000000..9870dbe --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/StatementCustomizer.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.customization; + +import org.apache.flink.annotation.PublicEvolving; + +import com.datastax.driver.core.Statement; + +import java.io.Serializable; + +/** + * Customizes Cassandra statements after they are prepared and bound but before execution. + * + *

This interface allows for per-statement customization of runtime properties that cannot be set + * in the CQL text itself. StatementCustomizer operates on individual bound statements at execution + * time and has access to the original record for record-based decisions. + * + *

Common Use Cases: + * + *

    + *
  • Record-based consistency: Different consistency levels based on record + * content + *
  • Dynamic idempotency: Mark operations idempotent based on record type + *
  • Conditional timeouts: Adjust timeouts based on record size or priority + *
  • Record-based routing: Set routing key based on record fields + *
+ * + *

Example Usage: + * + *

{@code
+ * // Example 1: Simple idempotent flag for all statements
+ * StatementCustomizer simpleCustomizer = (statement, record) -> {
+ *     statement.setIdempotent(true);
+ * };
+ *
+ * // Example 2: Record-based consistency and timeouts
+ * StatementCustomizer orderCustomizer = (statement, order) -> {
+ *     if (order.getPriority() == Priority.CRITICAL) {
+ *         statement.setConsistencyLevel(ConsistencyLevel.QUORUM);
+ *         statement.setReadTimeoutMillis(5000);
+ *     } else {
+ *         statement.setConsistencyLevel(ConsistencyLevel.ONE);
+ *         statement.setReadTimeoutMillis(2000);
+ *     }
+ *     if (order.getItemCount() > 100) {
+ *         statement.setReadTimeoutMillis(10000);
+ *     }
+ *
+ *     statement.setIdempotent(true);
+ * };
+ *
+ * }
+ * + *

The customizer is applied after value binding, giving access to both the final statement and + * the original record that generated it. + * + * @param the input record type + */ +@PublicEvolving +@FunctionalInterface +public interface StatementCustomizer extends Serializable { + + /** + * Applies customizations to the bound statement before execution. + * + * @param statement the bound statement ready for execution + * @param record the original input record that generated this statement + */ + void apply(Statement statement, INPUT record); +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/ClauseBindings.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/ClauseBindings.java new file mode 100644 index 0000000..d65fe4a --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/ClauseBindings.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.modifiers; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * Holds bind values for CQL clause modifiers (USING and IF clauses). + * + *

This class separates USING clause values (TTL, TIMESTAMP) from IF clause values to ensure + * correct bind order in prepared statements. + */ +@PublicEvolving +public final class ClauseBindings implements Serializable { + private static final long serialVersionUID = 1L; + + private static final ClauseBindings EMPTY = new ClauseBindings(new Object[0], new Object[0]); + + private final Object[] usingValues; // TTL, TIMESTAMP values + private final Object[] ifValues; // IF condition values + + public ClauseBindings(Object[] usingValues, Object[] ifValues) { + this.usingValues = usingValues != null ? usingValues : new Object[0]; + this.ifValues = ifValues != null ? ifValues : new Object[0]; + } + + public Object[] getUsingValues() { + return usingValues; + } + + public Object[] getIfValues() { + return ifValues; + } + + public static ClauseBindings empty() { + return EMPTY; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/CqlClauseResolver.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/CqlClauseResolver.java new file mode 100644 index 0000000..9a04893 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/CqlClauseResolver.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.modifiers; + +import org.apache.flink.annotation.PublicEvolving; + +import com.datastax.driver.core.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.Update; + +import java.io.Serializable; + +/** + * Resolver for CQL clauses that must be part of the query text. + * + *

This interface handles CQL clauses that must be included in the query string during + * preparation time, such as: + * + *

    + *
  • USING TTL <expr> - can use bind markers for dynamic values (INSERT only) + *
  • USING TIMESTAMP <expr> - can use bind markers for dynamic values (UPDATE only) + *
  • IF NOT EXISTS (INSERT only) + *
  • IF EXISTS (UPDATE only) + *
  • IF <condition> - can use bind markers for condition values + *
  • Other USING clauses that are part of CQL syntax + *
+ * + *

Bind Marker Support: When using bind markers (placeholders) in clauses, + * return the corresponding values to ensure proper binding at execution time. Each bind marker + * (`?`) added to the query must have a corresponding value in the returned array. + * + *

Note: The examples use static imports from {@code + * com.datastax.driver.core.querybuilder.QueryBuilder} for methods like {@code ttl()}, {@code + * timestamp()}, {@code bindMarker()}, and {@code eq()}. + * + * @param the input record type + */ +@PublicEvolving +public interface CqlClauseResolver extends Serializable { + + /** + * Apply USING/IF clauses to an INSERT statement and return bind values. + * + *

INSERT statements only support USING clauses (TTL, TIMESTAMP) and IF NOT EXISTS. The IF + * NOT EXISTS clause doesn't require bind values. + * + *

Example: Dynamic TTL + * + *

{@code
+     * import static com.datastax.driver.core.querybuilder.QueryBuilder.*;
+     *
+     * public ClauseBindings applyTo(Insert insert, MyRecord record) {
+     *     insert.using(ttl(bindMarker()));
+     *     return new ClauseBindings(new Object[]{record.getTtl()}, null);
+     * }
+     * }
+ * + * @param insert the INSERT statement to modify + * @param record the input record that may contain clause values + * @return ClauseBindings with usingValues populated (ifValues will be empty for INSERT) + */ + default ClauseBindings applyTo(Insert insert, T record) { + return ClauseBindings.empty(); + } + + /** + * Apply USING/IF clauses to an UPDATE statement and return bind values. + * + *

UPDATE statements support both USING clauses (TTL, TIMESTAMP) and IF clauses (conditions). + * USING clauses appear before SET in the query, IF clauses appear after WHERE. + * + *

Example 1: USING clause only + * + *

{@code
+     * import static com.datastax.driver.core.querybuilder.QueryBuilder.*;
+     *
+     * public ClauseBindings applyTo(Update update, MyRecord record) {
+     *     update.using(ttl(bindMarker()));
+     *     return new ClauseBindings(new Object[]{3600}, null);
+     * }
+     * }
+ * + *

Example 2: IF clause only + * + *

{@code
+     * import static com.datastax.driver.core.querybuilder.QueryBuilder.*;
+     *
+     * public ClauseBindings applyTo(Update update, MyRecord record) {
+     *     update.onlyIf(eq("version", bindMarker()));
+     *     return new ClauseBindings(null, new Object[]{record.getExpectedVersion()});
+     * }
+     * }
+ * + *

Example 3: Both USING and IF clauses + * + *

{@code
+     * import static com.datastax.driver.core.querybuilder.QueryBuilder.*;
+     *
+     * public ClauseBindings applyTo(Update update, MyRecord record) {
+     *     update.using(ttl(bindMarker()))
+     *           .and(timestamp(bindMarker()))
+     *           .onlyIf(eq("version", bindMarker()));
+     *     return new ClauseBindings(
+     *         new Object[]{7200, 1234567890L},  // USING values
+     *         new Object[]{5}                    // IF values
+     *     );
+     * }
+     * }
+ * + * @param update the UPDATE statement to modify + * @param record the input record that may contain clause values + * @return ClauseBindings with separated USING and IF values + */ + default ClauseBindings applyTo(Update update, T record) { + return ClauseBindings.empty(); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/NoOpClauseResolver.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/NoOpClauseResolver.java new file mode 100644 index 0000000..0c9e64a --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/modifiers/NoOpClauseResolver.java @@ -0,0 +1,4 @@ +package org.apache.flink.connector.cassandra.sink.planner.core.modifiers; + +/** A no-op implementation of {@link CqlClauseResolver} that adds no CQL clauses. */ +public class NoOpClauseResolver implements CqlClauseResolver {} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/resolution/ResolutionMode.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/resolution/ResolutionMode.java new file mode 100644 index 0000000..e5c6c82 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/resolution/ResolutionMode.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.resolution; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Defines how CQL statements handle value bindings and field resolution. + * + *

This enum controls whether field positions and values are resolved at runtime (DYNAMIC) or + * pre-computed (STATIC). It affects both query generation and value binding in Cassandra sink + * operations. + */ +@PublicEvolving +public enum ResolutionMode { + + /** Indicates configuration hasn't been set yet. */ + UNSET, + + /** + * Dynamic resolution mode - field positions and values are resolved at runtime. + * + *

In this mode: + * + *

    + *
  • CQL statements use bind markers (?) for all fields + *
  • Field positions are computed for each record during binding + *
+ * + *

Example: + * + *

+     * INSERT INTO users (id, name, age) VALUES (?, ?, ?)
+     * // Field positions resolved at runtime for each record
+     * 
+ */ + DYNAMIC, + + /** + * Static resolution mode - field positions and values are pre-computed. + * + *

In this mode: + * + *

    + *
  • CQL statements may have literal values for key fields + *
  • Field positions are pre-computed during query parsing + *
+ * + *

Example: + * + *

+     * UPDATE users SET name = ?, age = ? WHERE id = 'user123'
+     * // Key field 'id' has literal value, positions pre-computed
+     * 
+ */ + STATIC +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/InsertStrategy.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/InsertStrategy.java new file mode 100644 index 0000000..45e1970 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/InsertStrategy.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.strategy; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.util.Preconditions; + +import com.datastax.driver.core.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.QueryBuilder; + +/** + * Strategy for generating INSERT statements in dynamic mode. + * + *

This strategy generates CQL INSERT statements from resolved write operations. It only handles + * INSERT operations and expects the {@code ResolvedWrite} to contain SET columns and values (WHERE + * columns should be empty). + * + *

Example generated query: + * + *

+ * INSERT INTO keyspace.table (id, name, email) VALUES (?, ?, ?)
+ * 
+ * + *

With modifiers: + * + *

+ * INSERT INTO keyspace.table (id, name, email) VALUES (?, ?, ?) USING TTL ? TIMESTAMP ?
+ * 
+ * + *

Value Binding Order: Values are bound in the following order: + * + *

    + *
  1. Column values (in the order of columns) + *
  2. Clause values (from CqlClauseResolver, if any) + *
+ * + * @param the input record type + */ +@Internal +public final class InsertStrategy implements PlannerStrategy { + + @Override + public QueryWithBindings getQueryWithBindings( + TableRef table, + CqlSinkConfig config, + CqlClauseResolver clauseResolver, + ResolvedWrite write, + T record) { + Preconditions.checkArgument(!write.isUpdate(), "InsertStrategy received UPDATE: " + write); + Insert insert = QueryBuilder.insertInto(table.keyspace(), table.tableName()); + for (String col : write.setColumns()) { + insert.value(col, QueryBuilder.bindMarker()); + } + ClauseBindings clauseBindings = clauseResolver.applyTo(insert, record); + return new QueryWithBindings(insert.getQueryString(), clauseBindings); + } + + @Override + public Object[] getOrderedBindValues(ResolvedWrite write, ClauseBindings clauseBindings) { + Object[] set = write.setValues(); + Object[] usingVals = clauseBindings.getUsingValues(); + if (usingVals.length == 0) { + return set; + } + Object[] out = new Object[set.length + usingVals.length]; + System.arraycopy(set, 0, out, 0, set.length); + System.arraycopy(usingVals, 0, out, set.length, usingVals.length); + return out; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/PlannerStrategy.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/PlannerStrategy.java new file mode 100644 index 0000000..82eb4fa --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/PlannerStrategy.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.strategy; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; + +import java.io.Serializable; + +/** + * Strategy interface for generating CQL queries from {@link ResolvedWrite} operations. + * + *

Each strategy is specialized for one operation type: + * + *

    + *
  • InsertStrategy: generates INSERT statements from resolved writes + *
  • UpdateStrategy: generates UPDATE statements from resolved writes + *
  • StaticInsertStrategy/StaticUpdateStrategy: returns user-provided queries as-is + *
+ * + * @param the input record type + */ +@Internal +public interface PlannerStrategy extends Serializable { + + /** + * Holds a built CQL query and any clause-originated bind values. + * + *

This class separates the complete CQL query string from values needed for CQL clauses + * (TTL, timestamp, IF conditions). Note: + * + *

    + *
  • Column values come from {@link + * org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver} + *
  • Clause values come from {@link CqlClauseResolver} (TTL, timestamp, conditions) + *
+ * + *

Examples: + * + *

Simple INSERT (no clauses): + * + *

{@code
+     * query: "INSERT INTO users (id, name, age) VALUES (?, ?, ?)"
+     * clauseBindings: []  // No TTL, timestamp, or conditions
+     * // Final bind order: [id_value, name_value, age_value]
+     * }
+ * + *

INSERT with TTL: + * + *

{@code
+     * query: "INSERT INTO users (id, name, age) VALUES (?, ?, ?) USING TTL ?"
+     * clauseBindings: [3600]  // TTL value from CqlClauseResolver
+     * // Final bind order: [id_value, name_value, age_value, 3600]
+     * }
+ * + *

UPDATE with TTL and Timestamp: + * + *

{@code
+     * query: "UPDATE users USING TTL ? AND TIMESTAMP ? SET name=?, age=? WHERE id=?"
+     * clauseBindings: [7200, 1234567890]  // TTL and timestamp from CqlClauseResolver
+     * // Final bind order: [7200, 1234567890, name_value, age_value, id_value]
+     * }
+ * + *

Conditional UPDATE with IF: + * + *

{@code
+     * query: "UPDATE users SET age=? WHERE id=? IF age=?"
+     * clauseBindings: [30]  // Expected age for IF condition
+     * // Final bind order: [age_value, id_value, 30]
+     * }
+ * + *

The {@link PlannerStrategy} method merges column values with clause values in the correct + * order for the prepared statement. + */ + final class QueryWithBindings implements Serializable { + public final String query; + public final ClauseBindings clauseBindings; + + public QueryWithBindings(String query, ClauseBindings clauseBindings) { + this.query = query; + this.clauseBindings = clauseBindings == null ? ClauseBindings.empty() : clauseBindings; + } + } + + /** + * Build query text and collect clause bindings introduced by clauseResolver (if any). + * + * @param table the target table information + * @param config the sink configuration + * @param clauseResolver resolver for applying CQL modifiers (TTL, timestamp, IF conditions) + * @param write the resolved write operation containing columns and values + * @param record the input record for per-record modifier resolution + * @return the query string and any clause bindings + */ + QueryWithBindings getQueryWithBindings( + TableRef table, + CqlSinkConfig config, + CqlClauseResolver clauseResolver, + ResolvedWrite write, + INPUT record); + + /** + * Get all bind values in the correct order for the prepared statement. + * + *

This method combines column values from ResolvedWrite with clause values from + * getQueryWithBindings() in the exact order required by the prepared statement. + * + *

Note: The bind order depends on the query structure: + * + *

    + *
  • INSERT: column values, then USING clause values (TTL, timestamp) + *
  • UPDATE with USING: USING values first, then SET, then WHERE, then IF values + *
  • UPDATE without USING: SET values, WHERE values, then IF clause values + *
+ * + * @param write the resolved write operation + * @param clauseBindings clause bindings from getQueryWithBindings() method + * @return array of all values to bind to the prepared statement + */ + Object[] getOrderedBindValues(ResolvedWrite write, ClauseBindings clauseBindings); +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/StaticInsertStrategy.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/StaticInsertStrategy.java new file mode 100644 index 0000000..ddc8ae2 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/StaticInsertStrategy.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.strategy; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.util.Preconditions; + +/** + * Static strategy for INSERT queries that returns the user-provided query string exactly as + * configured. + * + *

This strategy is used when users provide their own INSERT query string. The strategy validates + * that the ResolvedWrite is indeed an INSERT operation and returns the exact query string provided + * by the user in the configuration. + * + *

Note: In static mode, any USING/IF clauses must be part of the user-provided query string. The + * CqlClauseResolver is ignored since we cannot modify the user's query. + * + * @param the input record type + */ +@Internal +public final class StaticInsertStrategy implements PlannerStrategy { + + @Override + public QueryWithBindings getQueryWithBindings( + TableRef table, + CqlSinkConfig config, + CqlClauseResolver clauseResolver, + ResolvedWrite write, + T record) { + Preconditions.checkArgument(!write.isUpdate(), "StaticInsertStrategy got UPDATE: " + write); + return new QueryWithBindings(config.getQuery(), null); + } + + @Override + public Object[] getOrderedBindValues(ResolvedWrite write, ClauseBindings ignored) { + return write.setValues(); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/StaticUpdateStrategy.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/StaticUpdateStrategy.java new file mode 100644 index 0000000..8c297a5 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/StaticUpdateStrategy.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.strategy; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.util.Preconditions; + +/** + * Static strategy for UPDATE queries that returns the user-provided query string exactly as + * configured. + * + *

This strategy is used when users provide their own UPDATE query string. + * + *

Note: In static mode, any USING/IF clauses must be part of the user-provided query string. The + * CqlClauseResolver is ignored since we cannot modify the user's query. + * + * @param the input record type + */ +@Internal +public final class StaticUpdateStrategy implements PlannerStrategy { + + @Override + public QueryWithBindings getQueryWithBindings( + TableRef table, + CqlSinkConfig config, + CqlClauseResolver cqlClauseResolver, + ResolvedWrite write, + T record) { + + Preconditions.checkArgument(write.isUpdate(), "StaticUpdateStrategy got INSERT: " + write); + return new QueryWithBindings(config.getQuery(), null); + } + + @Override + public Object[] getOrderedBindValues(ResolvedWrite write, ClauseBindings ignored) { + Object[] set = write.setValues(); + Object[] where = write.whereValues(); + Object[] out = new Object[set.length + where.length]; + System.arraycopy(set, 0, out, 0, set.length); + System.arraycopy(where, 0, out, set.length, where.length); + return out; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/UpdateStrategy.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/UpdateStrategy.java new file mode 100644 index 0000000..6f5338d --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/UpdateStrategy.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.strategy; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.util.Preconditions; + +import com.datastax.driver.core.querybuilder.QueryBuilder; +import com.datastax.driver.core.querybuilder.Update; + +/** + * Strategy for generating UPDATE statements in dynamic mode. + * + *

This strategy generates CQL UPDATE statements from resolved write operations. It only handles + * UPDATE operations and expects the {@code ResolvedWrite} to contain both SET columns/values and + * WHERE columns/values. + * + *

Example generated query: + * + *

+ * UPDATE keyspace.table SET name = ?, email = ? WHERE id = ?
+ * 
+ * + *

With modifiers: + * + *

+ * UPDATE keyspace.table USING TTL 3600 SET name = ?, email = ? WHERE id = ?
+ * 
+ * + *

Value Binding Order: Values are bound in the following order: + * + *

    + *
  1. SET values (in the order of SET columns) + *
  2. WHERE values (in the order of WHERE columns) + *
  3. Clause values (from CqlClauseResolver, if any) + *
+ * + * @param the input record type + */ +@Internal +public final class UpdateStrategy implements PlannerStrategy { + + @Override + public QueryWithBindings getQueryWithBindings( + TableRef table, + CqlSinkConfig config, + CqlClauseResolver cqlClauseResolver, + ResolvedWrite write, + T record) { + Preconditions.checkArgument(write.isUpdate(), "UpdateStrategy received INSERT: " + write); + Update update = QueryBuilder.update(table.keyspace(), table.tableName()); + // Add SET clauses + for (String col : write.setColumns()) { + update.with(QueryBuilder.set(col, QueryBuilder.bindMarker())); + } + // Add WHERE clauses + for (String wc : write.whereColumns()) { + update.where(QueryBuilder.eq(wc, QueryBuilder.bindMarker())); + } + ClauseBindings clauseBindings = cqlClauseResolver.applyTo(update, record); + return new QueryWithBindings(update.getQueryString(), clauseBindings); + } + + @Override + public Object[] getOrderedBindValues(ResolvedWrite write, ClauseBindings clauseBindings) { + Object[] set = write.setValues(); + Object[] where = write.whereValues(); + Object[] usingVals = clauseBindings.getUsingValues(); + Object[] ifVals = clauseBindings.getIfValues(); + + Object[] out = new Object[set.length + where.length + usingVals.length + ifVals.length]; + int p = 0; + + // USING values come first in UPDATE queries + if (usingVals.length > 0) { + System.arraycopy(usingVals, 0, out, p, usingVals.length); + p += usingVals.length; + } + + // Then SET values + System.arraycopy(set, 0, out, p, set.length); + p += set.length; + + // Then WHERE values + System.arraycopy(where, 0, out, p, where.length); + p += where.length; + + // Finally IF values + if (ifVals.length > 0) { + System.arraycopy(ifVals, 0, out, p, ifVals.length); + } + + return out; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ColumnValueResolver.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ColumnValueResolver.java new file mode 100644 index 0000000..ddccc25 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ColumnValueResolver.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * Resolves write operations (INSERT or UPDATE) for CQL-based records. + * + *

This abstraction allows for dynamic determination of whether to perform an INSERT or UPDATE + * operation, along with the appropriate columns and values for each operation type. + * + *

Example Use Cases: + * + *

{@code
+ * // Example 1: INSERT operation with Row
+ * ColumnValueResolver insertResolver = record -> {
+ *     List columns = Arrays.asList("id", "name", "email");
+ *     Object[] values = new Object[] {
+ *         record.getField(0),  // id
+ *         record.getField(1),  // name
+ *         record.getField(2)   // email
+ *     };
+ *     return ResolvedWrite.insert(columns, values);
+ * };
+ *
+ * // Example 2: UPDATE operation with Row
+ * // Note: UPDATE requires non-empty SET clause and non-empty WHERE clause
+ * ColumnValueResolver updateResolver = record -> {
+ *     List setColumns = Arrays.asList("name", "email");
+ *     Object[] setValues = new Object[] {
+ *         record.getField(1),  // name
+ *         record.getField(2)   // email
+ *     };
+ *     List whereColumns = Arrays.asList("id");
+ *     Object[] whereValues = new Object[] {
+ *         record.getField(0)   // id
+ *     };
+ *     return ResolvedWrite.update(setColumns, setValues, whereColumns, whereValues);
+ * };
+ *
+ * // Example 3: INSERT operation with POJO
+ * public class User {
+ *     private int id;
+ *     private String name;
+ *     private String email;
+ *     // getters and setters...
+ * }
+ *
+ * ColumnValueResolver pojoInsertResolver = user -> {
+ *     List columns = Arrays.asList("id", "name", "email");
+ *     Object[] values = new Object[] {
+ *         user.getId(),
+ *         user.getName(),
+ *         user.getEmail()
+ *     };
+ *     return ResolvedWrite.insert(columns, values);
+ * };
+ *
+ * // Example 4: UPDATE operation with POJO
+ * ColumnValueResolver pojoUpdateResolver = user -> {
+ *     List setColumns = Arrays.asList("name", "email");
+ *     Object[] setValues = new Object[] {
+ *         user.getName(),
+ *         user.getEmail()
+ *     };
+ *     List whereColumns = Arrays.asList("id");
+ *     Object[] whereValues = new Object[] {
+ *         user.getId()
+ *     };
+ *     return ResolvedWrite.update(setColumns, setValues, whereColumns, whereValues);
+ * };
+ *
+ * // Example 5: Conditional INSERT vs UPDATE based on record content
+ * ColumnValueResolver conditionalResolver = record -> {
+ *     boolean isUpdate = record.getField(3) != null;  // has an update flag
+ *
+ *     if (isUpdate) {
+ *         return ResolvedWrite.update(
+ *             Arrays.asList("name", "email"),
+ *             new Object[]{record.getField(1), record.getField(2)},
+ *             Arrays.asList("id"),
+ *             new Object[]{record.getField(0)}
+ *         );
+ *     } else {
+ *         return ResolvedWrite.insert(
+ *             Arrays.asList("id", "name", "email"),
+ *             new Object[]{record.getField(0), record.getField(1), record.getField(2)}
+ *         );
+ *     }
+ * };
+ * }
+ * + *

When to Use: Use ColumnValueResolver in DYNAMIC mode when you need + * record-specific operation determination, conditional INSERT/UPDATE logic, field transformation, + * or computed columns that cannot be expressed with static configuration. + * + * @param the input record type + */ +@PublicEvolving +public interface ColumnValueResolver extends Serializable { + + /** Enum defining the type of CQL operation this resolver handles. */ + @PublicEvolving + enum Kind { + INSERT, + UPDATE + } + + /** + * Returns the kind of operation this resolver handles. + * + * @return the operation kind (INSERT or UPDATE) + */ + Kind kind(); + + /** + * Resolves a write operation for the given record. + * + *

The returned ResolvedWrite should match the kind() returned by this resolver. + * + *

UPDATE Constraints: For UPDATE operations, the SET clause must contain at + * least one column-value pair (non-empty), and the WHERE clause must contain at least one + * column-value pair (non-empty) to identify the row(s) to update. + * + * @param record the input record + * @return the resolved write operation with columns and values + */ + ResolvedWrite resolve(INPUT record); +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedColumnValueResolver.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedColumnValueResolver.java new file mode 100644 index 0000000..647e73e --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedColumnValueResolver.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.cassandra.sink.config.RecordFormatType; +import org.apache.flink.connector.cassandra.sink.util.CqlStatementHelper; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Internal resolver for static queries that extracts fields in parameter marker order and returns + * ResolvedWrite based on the query type. + * + *

This resolver is used internally when static queries are parsed to determine the parameter + * order. It extracts values from records in the order they appear as parameter markers in the + * static query. + * + * @param the input record type + */ +@Internal +public final class FixedColumnValueResolver implements ColumnValueResolver { + + private final RecordFormatType recordFormatType; + private final List setColumns; + private final List whereColumns; + private final Kind kind; + + /** + * Creates a static resolver for INSERT operations (no WHERE columns). + * + * @param recordFormatType the record format for field extraction + * @param setColumns the columns in parameter marker order + */ + public FixedColumnValueResolver(RecordFormatType recordFormatType, List setColumns) { + this(recordFormatType, setColumns, Collections.emptyList()); + } + + /** + * Creates a static resolver for INSERT or UPDATE operations. + * + * @param recordFormatType the record format for field extraction + * @param setColumns the SET columns in parameter marker order + * @param whereColumns the WHERE columns in parameter marker order (empty for INSERT) + */ + public FixedColumnValueResolver( + RecordFormatType recordFormatType, List setColumns, List whereColumns) { + this.recordFormatType = recordFormatType; + this.setColumns = setColumns; + this.whereColumns = whereColumns; + this.kind = whereColumns.isEmpty() ? Kind.INSERT : Kind.UPDATE; + } + + @Override + public Kind kind() { + return kind; + } + + @Override + public ResolvedWrite resolve(INPUT record) { + // Extract all fields from record - assumes record fields match parameter marker order + Object[] allValues = CqlStatementHelper.extractFields(record, recordFormatType); + ResolvedWrite write = null; + switch (kind) { + case INSERT: + // INSERT operation - all values go to SET clause + write = ResolvedWrite.insert(setColumns, allValues); + break; + case UPDATE: + // UPDATE operation - split values between SET and WHERE clauses + int setCount = setColumns.size(); + int expectedTotal = getExpectedTotal(setCount, allValues); + + Object[] setValues = Arrays.copyOfRange(allValues, 0, setCount); + Object[] whereValues = Arrays.copyOfRange(allValues, setCount, expectedTotal); + + write = ResolvedWrite.update(setColumns, setValues, whereColumns, whereValues); + } + return write; + } + + private int getExpectedTotal(int setCount, Object[] allValues) { + int whereCount = whereColumns.size(); + int expectedTotal = setCount + whereCount; + + // Validate we have enough fields before splitting + if (allValues.length < expectedTotal) { + throw new IllegalStateException( + String.format( + "UPDATE parameter count mismatch: query expects %d parameters (%d SET + %d WHERE) but record has %d fields", + expectedTotal, setCount, whereCount, allValues.length)); + } + return expectedTotal; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedTableResolver.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedTableResolver.java new file mode 100644 index 0000000..6d5bf36 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedTableResolver.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.apache.flink.annotation.Internal; + +/** + * A table resolver that always returns the same fixed table. This is the default behavior for most + * use cases. (Used when static CQl is provided.) + * + * @param the input record type + */ +@Internal +public class FixedTableResolver implements TableResolver { + + private final TableRef tableRef; + + /** + * Creates a fixed table resolver. + * + * @param keyspace the keyspace name + * @param tableName the table name + */ + public FixedTableResolver(String keyspace, String tableName) { + this.tableRef = new TableRef(keyspace, tableName); + } + + /** + * Creates a fixed table resolver. + * + * @param tableRef the table reference + */ + public FixedTableResolver(TableRef tableRef) { + this.tableRef = tableRef; + } + + @Override + public TableRef resolve(Input record) { + return tableRef; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ResolvedWrite.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ResolvedWrite.java new file mode 100644 index 0000000..0468ab3 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ResolvedWrite.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Resolved write operation containing columns and values for INSERT or UPDATE statements. + * + *

This class represents a resolved write operation that can describe either an INSERT or UPDATE. + * The resolver determines everything about what to write, and the strategy just builds the + * appropriate CQL query. + * + *

For INSERT operations:

+ * + *
    + *
  • Use {@link #insert(List, Object[])} factory method + *
  • Provides only column names and values to insert + *
  • {@code whereColumns} and {@code whereValues} will be empty + *
  • Will generate CQL like: {@code INSERT INTO table (id, name, age) VALUES (?, ?, ?)} + *
  • Bind order: values in the same order as columns + *
+ * + *

For UPDATE operations:

+ * + *
    + *
  • Use {@link #update(List, Object[], List, Object[])} factory method + *
  • {@code setColumns}/{@code setValues}: columns to update with new values + *
  • {@code whereColumns}/{@code whereValues}: primary key columns for WHERE clause + *
  • Will generate CQL like: {@code UPDATE table SET name=?, age=? WHERE id=?} + *
  • Bind order: SET values first, then WHERE values + *
  • Important: Columns cannot appear in both SET and WHERE (enforced by + * validation) + *
+ * + *

Example Usage:

+ * + *
{@code
+ * // INSERT operation - all columns go into the INSERT
+ * ResolvedWrite insert = ResolvedWrite.insert(
+ *     Arrays.asList("id", "name", "age"),
+ *     new Object[]{101, "Alice", 30}
+ * );
+ * // Generates: INSERT INTO table (id, name, age) VALUES (?, ?, ?)
+ * // Binds: [101, "Alice", 30]
+ *
+ * // UPDATE operation - separate SET and WHERE clauses
+ * ResolvedWrite update = ResolvedWrite.update(
+ *     Arrays.asList("name", "age"),           // SET columns
+ *     new Object[]{"Alice Updated", 31},      // SET values
+ *     Arrays.asList("id"),                    // WHERE columns (primary key)
+ *     new Object[]{101}                       // WHERE values
+ * );
+ * // Generates: UPDATE table SET name=?, age=? WHERE id=?
+ * // Binds: ["Alice Updated", 31, 101] (SET values first, then WHERE values)
+ * }
+ */ +@PublicEvolving +public final class ResolvedWrite implements Serializable { + + private static final long serialVersionUID = 1L; + + private final List setColumns; + private final Object[] setValues; + private final List whereColumns; + private final Object[] whereValues; + + ResolvedWrite( + List setColumns, + Object[] setValues, + List whereColumns, + Object[] whereValues) { + // Validate non-null parameters + this.setColumns = Preconditions.checkNotNull(setColumns, "setColumns cannot be null"); + this.setValues = Preconditions.checkNotNull(setValues, "setValues cannot be null"); + this.whereColumns = Preconditions.checkNotNull(whereColumns, "whereColumns cannot be null"); + this.whereValues = Preconditions.checkNotNull(whereValues, "whereValues cannot be null"); + + // Validate equal lengths + Preconditions.checkArgument( + setColumns.size() == setValues.length, + "setColumns size (%s) must equal setValues length (%s)", + setColumns.size(), + setValues.length); + Preconditions.checkArgument( + whereColumns.size() == whereValues.length, + "whereColumns size (%s) must equal whereValues length (%s)", + whereColumns.size(), + whereValues.length); + + // Validate at least one SET column for meaningful operations + Preconditions.checkArgument( + !setColumns.isEmpty(), "setColumns cannot be empty - need at least one column"); + + // Validate disjoint column names (SET and WHERE columns should not overlap) + if (!whereColumns.isEmpty()) { + Set setColumnSet = new HashSet<>(setColumns); + for (String whereColumn : whereColumns) { + Preconditions.checkArgument( + !setColumnSet.contains(whereColumn), + "Column '%s' cannot appear in both SET and WHERE clauses", + whereColumn); + } + } + } + + /** + * Factory method for creating an INSERT operation. + * + * @param setColumns the columns to insert into + * @param setValues the values to insert + * @return a ResolvedWrite representing an INSERT operation + */ + public static ResolvedWrite insert(List setColumns, Object[] setValues) { + return new ResolvedWrite(setColumns, setValues, Collections.emptyList(), new Object[0]); + } + + /** + * Factory method for creating an UPDATE operation. + * + * @param setColumns the columns to update (SET clause) + * @param setValues the values to set + * @param whereColumns the columns for the WHERE clause (typically primary key columns) + * @param whereValues the values for the WHERE clause + * @return a ResolvedWrite representing an UPDATE operation + */ + public static ResolvedWrite update( + List setColumns, + Object[] setValues, + List whereColumns, + Object[] whereValues) { + return new ResolvedWrite(setColumns, setValues, whereColumns, whereValues); + } + + /** + * Returns whether this is an UPDATE operation. + * + * @return true if this represents an UPDATE operation (has WHERE columns), false if INSERT + */ + public boolean isUpdate() { + return !whereColumns.isEmpty(); + } + + /** + * Returns the columns for the SET clause (INSERT columns for INSERT operations). + * + * @return list of column names for SET clause + */ + public List setColumns() { + return setColumns; + } + + /** + * Returns the values for the SET clause (INSERT values for INSERT operations). + * + * @return array of values for SET clause + */ + public Object[] setValues() { + return setValues; + } + + /** + * Returns the columns for the WHERE clause (empty for INSERT operations). + * + * @return list of column names for WHERE clause + */ + public List whereColumns() { + return whereColumns; + } + + /** + * Returns the values for the WHERE clause (empty for INSERT operations). + * + * @return array of values for WHERE clause + */ + public Object[] whereValues() { + return whereValues; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableRef.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableRef.java new file mode 100644 index 0000000..fb73461 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableRef.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.StringUtils; + +import java.io.Serializable; + +/** Reference to a Cassandra table consisting of keyspace and table name. */ +@PublicEvolving +public class TableRef implements Serializable { + + private static final long serialVersionUID = 1L; + + private final String keyspace; + private final String tableName; + + public TableRef(String keyspace, String tableName) { + Preconditions.checkArgument( + keyspace != null && !StringUtils.isEmpty(keyspace.trim()), + "keyspace cannot be null/empty"); + Preconditions.checkArgument( + tableName != null && !StringUtils.isEmpty(tableName.trim()), + "tableName cannot be null/empty"); + this.keyspace = keyspace; + this.tableName = tableName; + } + + public String keyspace() { + return keyspace; + } + + public String tableName() { + return tableName; + } + + /** Returns the fully qualified table name in format "keyspace.tableName". */ + public String getFullyQualifiedName() { + return keyspace + "." + tableName; + } + + @Override + public String toString() { + return getFullyQualifiedName(); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableResolver.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableResolver.java new file mode 100644 index 0000000..7903b3c --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableResolver.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * Resolves which Cassandra table to write a record to. + * + *

This abstraction allows for dynamic table selection based on record content, such as + * date-based table rotation, sharding strategies, or tenant-based routing. + * + *

Example Use Cases: + * + *

{@code
+ * // Example 1: Date-based table partitioning
+ * TableResolver dateBasedResolver = record -> {
+ *     Timestamp timestamp = (Timestamp) record.getField(0);
+ *     String dateStr = new SimpleDateFormat("yyyyMM").format(timestamp);
+ *     return new TableRef("analytics", "events_" + dateStr);
+ * };
+ *
+ * // Example 2: Tenant-based table routing
+ * TableResolver tenantResolver = record -> {
+ *     String tenantId = (String) record.getField(0);
+ *     return new TableRef("multi_tenant", "tenant_" + tenantId + "_data");
+ * };
+ *
+ * // Example 3: Record type-based routing
+ * TableResolver typeBasedResolver = record -> {
+ *     // Assuming type field is at index 2
+ *     String recordType = (String) record.getField(2);
+ *     switch (recordType) {
+ *         case "USER_EVENT":
+ *             return new TableRef("events", "user_events");
+ *         case "SYSTEM_EVENT":
+ *             return new TableRef("events", "system_events");
+ *         case "ERROR_EVENT":
+ *             return new TableRef("monitoring", "error_logs");
+ *         default:
+ *             return new TableRef("events", "unknown_events");
+ *     }
+ * };
+ *
+ * }
+ * + *

When to Use: Use TableResolver in DYNAMIC mode when you need to route records + * to different tables based on record content, time-based partitioning, tenant isolation, or + * sharding strategies. + * + * @param the input record type + */ +@PublicEvolving +@FunctionalInterface +public interface TableResolver extends Serializable { + + /** + * Determines the target table for the given record. + * + * @param record the input record + * @return the table reference containing keyspace and table name + */ + TableRef resolve(Input record); +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/CqlStatementHelper.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/CqlStatementHelper.java new file mode 100644 index 0000000..d70eed6 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/CqlStatementHelper.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.connector.cassandra.sink.config.RecordFormatType; +import org.apache.flink.types.Row; + +import com.datastax.driver.core.BoundStatement; +import com.datastax.driver.core.PreparedStatement; + +import javax.annotation.Nullable; + +import scala.Product; + +/** + * Helper utility for working with CQL insert statements. + * + *

This utility supports: + * + *

    + *
  • Extraction of values from structured record formats (Tuple, Row, Scala Product) + *
  • Binding extracted values into prepared Cassandra statements + *
  • Handling of null fields with optional suppression + *
+ */ +@Internal +public class CqlStatementHelper { + + /** + * Extracts bindable field values from a supported input record type. + * + * @param record the record to extract values from + * @param formatType the record format type + * @return an array of values to be bound to a prepared statement + * @throws IllegalArgumentException if the format type is unsupported + */ + public static Object[] extractFields(Object record, RecordFormatType formatType) { + switch (formatType) { + case TUPLE: + Tuple tuple = (Tuple) record; + return extractValues(tuple.getArity(), tuple::getField); + case ROW: + Row row = (Row) record; + return extractValues(row.getArity(), row::getField); + case SCALA_PRODUCT: + Product product = (Product) record; + return extractValues(product.productArity(), product::productElement); + default: + throw new IllegalArgumentException("Unsupported RecordFormatType: " + formatType); + } + } + + /** + * Binds values to a prepared statement. + * + * @param preparedStatement the prepared statement + * @param values the values to bind + * @return the bound statement ready for execution + */ + public static BoundStatement bind(PreparedStatement preparedStatement, Object[] values) { + return preparedStatement.bind(values); + } + + /** Helper method to extract values using a generic field accessor. */ + private static Object[] extractValues(int arity, FieldAccessor accessor) { + Object[] values = new Object[arity]; + for (int i = 0; i < arity; i++) { + values[i] = accessor.getField(i); + } + return values; + } + + /** Functional interface for accessing fields by index. */ + @FunctionalInterface + private interface FieldAccessor { + @Nullable + Object getField(int index); + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/QueryParser.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/QueryParser.java new file mode 100644 index 0000000..4da3cb2 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/QueryParser.java @@ -0,0 +1,483 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Utility class for parsing CQL INSERT and UPDATE queries to extract table information and column + * names. + * + *

This parser supports standard INSERT INTO and UPDATE statements with the following formats: + * + *

    + *
  • INSERT: {@code INSERT INTO keyspace.table (column1, column2, ...) VALUES (?, ?, ...)} + *
  • UPDATE: {@code UPDATE keyspace.table SET column1=?, column2=? WHERE id=? AND status IN (?, + * ?)} + *
+ * + *

The parser extracts keyspace, table, and column names from queries while ignoring USING and IF + * clauses which should contain literal values in static mode. + * + *

Supported query variations include: + * + *

    + *
  • Quoted identifiers: {@code "MixedCase", "user-profile", "order"} + *
  • Trailing clauses: {@code USING TTL 3600, IF NOT EXISTS} + *
  • Flexible whitespace and case variations + *
+ * + *

Example usage: + * + *

{@code
+ * String insertQuery = "INSERT INTO analytics.events (user_id, event_time, event_type) VALUES (?, ?, ?)";
+ * QueryInfo insertInfo = QueryParser.parseInsertQuery(insertQuery);
+ * // insertInfo.getKeyspace() = "analytics"
+ * // insertInfo.getTableName() = "events"
+ * // insertInfo.getColumnNames() = ["user_id", "event_time", "event_type"]
+ *
+ * String updateQuery = "UPDATE analytics.events SET event_type=?, last_seen=? WHERE user_id IN (?, ?, ?)";
+ * UpdateQueryInfo updateInfo = QueryParser.parseUpdateQuery(updateQuery);
+ * // updateInfo.getKeyspace() = "analytics"
+ * // updateInfo.getTableName() = "events"
+ * // updateInfo.getSetColumns() = ["event_type", "last_seen"]
+ * // updateInfo.getWhereColumns() = ["user_id"]
+ * }
+ */ +@Internal +public final class QueryParser { + + /** + * Regex pattern for CQL identifiers: unquoted (letter/underscore start) or quoted (with escaped + * quotes). + */ + private static final String IDENTIFIER = "(?:\"(?:[^\"]|\"\")+\"|[A-Za-z_][A-Za-z0-9_\\$]*)"; + + /** + * Pattern for INSERT queries - captures core structure while being permissive of trailing + * clauses. Groups: (1) keyspace, (2) table, (3) columns, (4) values placeholders + */ + private static final Pattern INSERT_PATTERN = + Pattern.compile( + "INSERT\\s+INTO\\s+(?:(" + + IDENTIFIER + + ")\\.)?(" + + IDENTIFIER + + ")\\s*" + + "\\(([^)]*)\\)\\s*VALUES\\s*\\(([^)]*)\\)", + Pattern.CASE_INSENSITIVE | Pattern.DOTALL); + + /** + * Pattern for UPDATE queries - captures SET and WHERE while allowing USING after table and IF + * at end. Groups: (1) keyspace, (2) table, (3) SET clause, (4) WHERE clause + */ + private static final Pattern UPDATE_PATTERN = + Pattern.compile( + "UPDATE\\s+(?:(" + + IDENTIFIER + + ")\\.)?(" + + IDENTIFIER + + ")\\s*" + + "(?:USING\\b.+?\\s+)?" // optional USING ... AFTER table + + "SET\\s+(.+?)\\s+WHERE\\s+(.+?)" // non-greedy SET/WHERE groups + + "(?:\\s+IF\\b.+)?\\s*$", // optional IF ... tail + Pattern.CASE_INSENSITIVE | Pattern.DOTALL); + + /** + * Pattern for WHERE conditions - matches column name followed by any operator and value. + * Groups: (1) column name, (2) operator and value part + */ + private static final Pattern WHERE_CONDITION_PATTERN = + Pattern.compile( + "^(" + IDENTIFIER + ")\\s*(.+)$", Pattern.CASE_INSENSITIVE | Pattern.DOTALL); + + /** Pattern for splitting WHERE conditions on AND (case-insensitive with flexible spacing). */ + private static final Pattern AND_SPLITTER = + Pattern.compile("\\s+AND\\s+", Pattern.CASE_INSENSITIVE); + + /** + * Parses a CQL INSERT query to extract table and column information. + * + * @param insertQuery the INSERT query to parse + * @return QueryInfo containing parsed keyspace, table, and column names + * @throws IllegalArgumentException if the query format is invalid + */ + public static QueryInfo parseInsertQuery(String insertQuery) { + Preconditions.checkArgument( + insertQuery != null && !StringUtils.isBlank(insertQuery), + "insertQuery cannot be null or blank"); + + Matcher matcher = INSERT_PATTERN.matcher(insertQuery); + Preconditions.checkArgument( + matcher.find(), + "Invalid INSERT query format. Expected 'INSERT INTO keyspace.table (columns) VALUES (placeholders)'. " + + "Static mode requires fully qualified keyspace.table. Got: %s", + insertQuery); + + String keyspace = dequote(matcher.group(1)); + String tableName = dequote(matcher.group(2)); + validateKeyspaceAndTable(keyspace, tableName, "INSERT"); + + String columnsText = matcher.group(3); + String valuesText = matcher.group(4); + + List columnNames = parseColumnNames(columnsText); + Preconditions.checkArgument( + !columnNames.isEmpty(), "At least one column must be specified"); + + List valuePlaceholders = parseValuePlaceholders(valuesText); + Preconditions.checkArgument( + columnNames.size() == valuePlaceholders.size(), + "Column count (%d) must match value placeholder count (%d). Columns: %s, Values: %s", + columnNames.size(), + valuePlaceholders.size(), + columnNames, + valuePlaceholders); + + return new QueryInfo(keyspace, tableName, columnNames); + } + + /** + * Parses a CQL UPDATE query to extract table and column information. + * + * @param updateQuery the UPDATE query to parse + * @return UpdateQueryInfo containing parsed keyspace, table, SET columns, and WHERE columns + * @throws IllegalArgumentException if the query format is invalid + */ + public static UpdateQueryInfo parseUpdateQuery(String updateQuery) { + Preconditions.checkArgument( + updateQuery != null && !StringUtils.isBlank(updateQuery), + "updateQuery cannot be null or blank"); + + Matcher matcher = UPDATE_PATTERN.matcher(updateQuery); + Preconditions.checkArgument( + matcher.find(), + "Invalid UPDATE query format. Expected 'UPDATE keyspace.table SET col1=?, col2=? WHERE id=?'. " + + "Static mode requires fully qualified keyspace.table. Got: %s", + updateQuery); + + String keyspace = dequote(matcher.group(1)); + String tableName = dequote(matcher.group(2)); + validateKeyspaceAndTable(keyspace, tableName, "UPDATE"); + + String setClause = matcher.group(3); + String whereClause = matcher.group(4); + List setColumns = parseSetClause(setClause); + Preconditions.checkArgument( + !setColumns.isEmpty(), "At least one SET column must be specified"); + List whereColumns = parseWhereClause(whereClause); + Preconditions.checkArgument( + !whereColumns.isEmpty(), "At least one WHERE column must be specified for UPDATE."); + + return new UpdateQueryInfo(keyspace, tableName, setColumns, whereColumns); + } + + /** + * Validates keyspace and table extracted from query. Example: validateKeyspaceAndTable(null, + * "users", "UPDATE") throws "Static mode requires fully qualified keyspace.table format" + */ + private static void validateKeyspaceAndTable( + String keyspace, String tableName, String queryType) { + Preconditions.checkArgument(!StringUtils.isEmpty(tableName), "Table name cannot be empty"); + Preconditions.checkArgument( + !StringUtils.isEmpty(keyspace), + "Static mode requires fully qualified keyspace.table format. " + + "Use '%s keyspace.table' instead of just 'table'", + queryType); + } + + /** + * De-quotes CQL identifiers if quoted, handling escaped quotes. Example: + * dequote("\"MixedCase\"") returns "MixedCase", dequote("\"has\"\"quote\"") returns + * "has\"quote" + */ + private static String dequote(String identifier) { + if (identifier != null + && identifier.length() >= 2 + && identifier.startsWith("\"") + && identifier.endsWith("\"")) { + return identifier.substring(1, identifier.length() - 1).replace("\"\"", "\""); + } + return identifier; + } + + /** + * Parses column names from comma-separated text, properly handling quoted identifiers with + * escaped quotes. Example: parseColumnNames("id, \"User-Name\", age") returns ["id", + * "User-Name", "age"] + */ + private static List parseColumnNames(String columnsText) { + if (StringUtils.isEmpty(columnsText)) { + return Collections.emptyList(); + } + + List columnNames = new ArrayList<>(); + StringBuilder currentToken = new StringBuilder(); + boolean insideQuotedIdentifier = false; + + for (int i = 0; i < columnsText.length(); i++) { + char currentChar = columnsText.charAt(i); + if (currentChar == '"') { + currentToken.append(currentChar); + // Check for doubled quote inside quotes (escaped quote) + if (insideQuotedIdentifier + && i + 1 < columnsText.length() + && columnsText.charAt(i + 1) == '"') { + currentToken.append('"'); + i++; // consume second quote + } else { + insideQuotedIdentifier = !insideQuotedIdentifier; + } + } else if (currentChar == ',' && !insideQuotedIdentifier) { + String columnName = currentToken.toString().trim(); + if (!columnName.isEmpty()) { + columnNames.add(dequote(columnName)); + } + currentToken.setLength(0); + } else { + currentToken.append(currentChar); + } + } + + // Add the last column + String lastColumnName = currentToken.toString().trim(); + if (!lastColumnName.isEmpty()) { + columnNames.add(dequote(lastColumnName)); + } + + return columnNames; + } + + /** + * Parses value placeholders from the VALUES section of an INSERT query, ensuring all are ?. + * Example: parseValuePlaceholders("?, ?, ?") returns ["?", "?", "?"], but "?, 'literal'" throws + * exception + */ + private static List parseValuePlaceholders(String valuesText) { + if (StringUtils.isEmpty(valuesText)) { + return Collections.emptyList(); + } + + String[] values = valuesText.split(","); + List trimmedValues = new ArrayList<>(); + + for (String value : values) { + String trimmed = value.trim(); + if (!trimmed.isEmpty()) { + Preconditions.checkArgument( + "?".equals(trimmed), + "Static mode supports only parameter placeholders (?) in VALUES clause. " + + "Literals and expressions are not supported. Found: '%s'", + trimmed); + trimmedValues.add(trimmed); + } + } + + return trimmedValues; + } + + /** + * Parses the SET clause of an UPDATE query to extract column names. Example: + * parseSetClause("name = ?, age = ?, \"Status\" = ?") returns ["name", "age", "Status"] + */ + private static List parseSetClause(String setClause) { + if (StringUtils.isEmpty(setClause)) { + return Collections.emptyList(); + } + + List setColumnNames = new ArrayList<>(); + StringBuilder currentAssignmentBuilder = new StringBuilder(); + boolean insideQuotedIdentifier = false; + int parenthesesNestingLevel = 0; + + for (char currentChar : setClause.toCharArray()) { + if (currentChar == '"' + && (currentAssignmentBuilder.length() == 0 + || currentAssignmentBuilder.charAt( + currentAssignmentBuilder.length() - 1) + != '"')) { + insideQuotedIdentifier = !insideQuotedIdentifier; + currentAssignmentBuilder.append(currentChar); + } else if (!insideQuotedIdentifier && currentChar == '(') { + parenthesesNestingLevel++; + currentAssignmentBuilder.append(currentChar); + } else if (!insideQuotedIdentifier && currentChar == ')') { + parenthesesNestingLevel--; + currentAssignmentBuilder.append(currentChar); + } else if (currentChar == ',' + && !insideQuotedIdentifier + && parenthesesNestingLevel == 0) { + String setAssignment = currentAssignmentBuilder.toString().trim(); + if (!setAssignment.isEmpty()) { + setColumnNames.add(parseSetAssignment(setAssignment)); + } + currentAssignmentBuilder = new StringBuilder(); + } else { + currentAssignmentBuilder.append(currentChar); + } + } + + // Add the last assignment + String finalSetAssignment = currentAssignmentBuilder.toString().trim(); + if (!finalSetAssignment.isEmpty()) { + setColumnNames.add(parseSetAssignment(finalSetAssignment)); + } + + return setColumnNames; + } + + /** + * Parses a single SET assignment and returns the column name. Example: + * parseSetAssignment("status = ?") returns "status", but "status = 'active'" throws exception + */ + private static String parseSetAssignment(String assignment) { + String[] parts = assignment.split("=", 2); + Preconditions.checkArgument( + parts.length == 2, + "Invalid SET assignment format. Expected 'column = ?'. Got: '%s'", + assignment); + + String column = parts[0].trim(); + String value = parts[1].trim(); + + Preconditions.checkArgument( + "?".equals(value), + "Static mode supports only parameter placeholders (?) in SET clause. " + + "Literals and expressions are not supported. Found: '%s'", + value); + + return dequote(column); + } + + /** + * Parses the WHERE clause of an UPDATE query to extract column names. Example: + * parseWhereClause("id = ? AND status IN (?, ?) AND age > ?") returns ["id", "status", "age"] + */ + private static List parseWhereClause(String whereClause) { + if (StringUtils.isEmpty(whereClause)) { + return Collections.emptyList(); + } + + List columns = new ArrayList<>(); + String[] conditions = AND_SPLITTER.split(whereClause.trim()); + + for (String condition : conditions) { + String trimmed = condition.trim(); + if (!trimmed.isEmpty()) { + columns.add(parseWhereCondition(trimmed)); + } + } + + return columns; + } + + /** + * Parses a single WHERE condition (supports =, !=, <, >, <=, >=, IN operators). Example: + * parseWhereCondition("id IN (?, ?, ?)") returns "id", parseWhereCondition("age > ?") returns + * "age" + */ + private static String parseWhereCondition(String condition) { + Matcher matcher = WHERE_CONDITION_PATTERN.matcher(condition.trim()); + Preconditions.checkArgument( + matcher.matches(), + "Invalid WHERE condition format. Expected 'column '. Got: '%s'", + condition); + + String column = dequote(matcher.group(1)); + String operatorAndValue = matcher.group(2); + + // Just verify that there's at least one placeholder in the value part + Preconditions.checkArgument( + operatorAndValue.contains("?"), + "WHERE condition must contain at least one parameter placeholder (?). " + + "Literals should be part of the query text in static mode. Found: '%s'", + condition); + + return column; + } + + /** Container for parsed query information. */ + public static class QueryInfo { + private final String keyspace; + private final String tableName; + private final List columnNames; + + public QueryInfo(String keyspace, String tableName, List columnNames) { + this.keyspace = keyspace; + this.tableName = tableName; + this.columnNames = Collections.unmodifiableList(new ArrayList<>(columnNames)); + } + + public String getKeyspace() { + return keyspace; + } + + public String getTableName() { + return tableName; + } + + public List getColumnNames() { + return columnNames; + } + } + + /** Container for parsed UPDATE query information. */ + public static class UpdateQueryInfo { + private final String keyspace; + private final String tableName; + private final List setColumns; + private final List whereColumns; + + public UpdateQueryInfo( + String keyspace, + String tableName, + List setColumns, + List whereColumns) { + this.keyspace = keyspace; + this.tableName = tableName; + this.setColumns = Collections.unmodifiableList(new ArrayList<>(setColumns)); + this.whereColumns = Collections.unmodifiableList(new ArrayList<>(whereColumns)); + } + + public String getKeyspace() { + return keyspace; + } + + public String getTableName() { + return tableName; + } + + public List getSetColumns() { + return setColumns; + } + + public List getWhereColumns() { + return whereColumns; + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/RecordWriterFactory.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/RecordWriterFactory.java new file mode 100644 index 0000000..8fea992 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/util/RecordWriterFactory.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.cassandra.sink.config.CassandraSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.PojoSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.RecordFormatType; +import org.apache.flink.connector.cassandra.sink.writer.CassandraRecordWriter; +import org.apache.flink.connector.cassandra.sink.writer.CqlRecordWriter; +import org.apache.flink.connector.cassandra.sink.writer.PojoRecordWriter; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; + +/** + * Factory for creating appropriate {@link CassandraRecordWriter} instances based on the sink + * configuration's record format type. + * + *

This factory uses the configuration's {@link RecordFormatType} to dispatch to the correct + * writer implementation without requiring runtime type checks or visitor patterns. + * + *

Supported mappings: + * + *

    + *
  • {@link RecordFormatType#POJO} → {@link PojoRecordWriter} (uses DataStax Mapper) + *
  • {@link RecordFormatType#TUPLE} → {@link CqlRecordWriter} (structured data) + *
  • {@link RecordFormatType#ROW} → {@link CqlRecordWriter} (structured data) + *
  • {@link RecordFormatType#SCALA_PRODUCT} → {@link CqlRecordWriter} (Scala types) + *
+ */ +@Internal +public final class RecordWriterFactory { + + /** + * Creates the appropriate record writer for the given configuration. + * + * @param clusterBuilder the cluster builder for Cassandra connection + * @param config the sink configuration + * @param the input record type + * @return a record writer appropriate for the configuration's format type + * @throws IllegalArgumentException if the format type is not supported + */ + public static CassandraRecordWriter create( + ClusterBuilder clusterBuilder, CassandraSinkConfig config) { + RecordFormatType formatType = config.getRecordFormatType(); + switch (formatType) { + case POJO: + return new PojoRecordWriter<>(clusterBuilder, (PojoSinkConfig) config); + case TUPLE: + case ROW: + case SCALA_PRODUCT: + return new CqlRecordWriter<>(clusterBuilder, (CqlSinkConfig) config); + default: + throw new IllegalArgumentException("Unsupported format type: " + formatType); + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/AbstractRecordWriter.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/AbstractRecordWriter.java new file mode 100644 index 0000000..b2a4dbf --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/AbstractRecordWriter.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.writer; + +import org.apache.flink.annotation.Internal; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.google.common.util.concurrent.ListenableFuture; + +/** + * Base class for {@link CassandraRecordWriter} implementations that provides: + * + *
    + *
  • Standard implementation of asynchronous statement execution + *
  • Graceful resource cleanup via {@link #close()} + *
+ * + *

Subclasses must provide access to a Cassandra {@link com.datastax.driver.core.Session} and + * {@link Cluster} by implementing {@link #getSession()} and {@link #getCluster()}. + * + * @param the input record type + */ +@Internal +public abstract class AbstractRecordWriter implements CassandraRecordWriter { + + /** + * Executes a statement asynchronously using the underlying session. + * + * @param statement the prepared or bound statement + * @return the future result of the execution + */ + @Override + public ListenableFuture executeStatement(Statement statement) { + return getSession().executeAsync(statement); + } + + /** + * Gets the Cassandra cluster instance. Used for resource cleanup. + * + * @return the cluster used by the writer + */ + protected abstract Cluster getCluster(); + + /** Closes the session and cluster if present. */ + @Override + public void close() { + final Session s = getSession(); + if (s != null) { + s.close(); + } + final Cluster c = getCluster(); + if (c != null) { + c.close(); + } + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/CassandraRecordWriter.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/CassandraRecordWriter.java new file mode 100644 index 0000000..30aaee8 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/CassandraRecordWriter.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.writer; + +import org.apache.flink.annotation.Internal; + +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.google.common.util.concurrent.ListenableFuture; + +/** + * An abstraction that defines how input records are converted into Cassandra writes. + * + *

Each implementation is responsible for: + * + *

    + *
  • Transforming a given input record into cassandra's {@link + * com.datastax.driver.core.BoundStatement} + *
  • Executing that statement against a live {@link Session} + *
  • Managing lifecycle of the Cassandra connection (via {@link #close()}) + *
+ * + * @param the type of input record written to Cassandra + */ +@Internal +public interface CassandraRecordWriter extends AutoCloseable { + + /** + * Returns the live Cassandra session used by this writer. + * + * @return the active Cassandra session + */ + Session getSession(); + + /** + * Converts a record into a fully bound Cassandra {@link Statement}. + * + *

“Fully bound” means there are no remaining {@code ?} bind markers on the returned + * statement. If the base query (or options such as TTL/TIMESTAMP) introduced N variables, the + * returned statement must have all N variables set (to a concrete value, to {@code null}, or + * marked as {@code UNSET} when supported by the protocol). + * + * @param input the input record + * @return a {@link Statement} ready for immediate execution + */ + Statement prepareStatement(INPUT input); + + /** + * Convenience method that performs both preparation and execution. + * + *

This method combines {@link #prepareStatement(Object)} and {@link + * #executeStatement(Statement)}. + * + * @param input the input record + * @return a {@link ListenableFuture} for the async operation + */ + default ListenableFuture write(INPUT input) { + return executeStatement(prepareStatement(input)); + } + + /** + * Executes a fully bound statement asynchronously using the writer's {@link Session}. + * + * @param statement the bound statement to execute + * @return a {@link ListenableFuture} representing the async result + */ + ListenableFuture executeStatement(Statement statement); + + /** + * Closes the writer and any internal Cassandra resources (session, cluster). + * + *

This method should be idempotent - safe to call multiple times. + */ + void close(); +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/CqlRecordWriter.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/CqlRecordWriter.java new file mode 100644 index 0000000..a78c4b4 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/CqlRecordWriter.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.writer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlannerFactory; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.util.Preconditions; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; + +/** + * A {@link CassandraRecordWriter} implementation that writes structured records (e.g., Tuple, Row, + * Scala Product) to Cassandra using explicit CQL insert queries. + * + *

This writer uses a user-provided or auto-generated {@link PreparedStatement} and binds + * extracted field values at runtime via {@link + * org.apache.flink.connector.cassandra.sink.util.CqlStatementHelper}. + * + *

The writer supports: + * + *

    + *
  • Field extraction based on the configured {@link + * org.apache.flink.connector.cassandra.sink.config.RecordFormatType} + *
  • Auto-generation of CQL INSERT statements from write options + *
  • Optional null field suppression to avoid tombstones + *
  • Query modifiers like TTL, timestamp, and consistency level + *
+ * + * @param the input record type (e.g., Tuple, Row, Scala Product) + */ +@Internal +public class CqlRecordWriter extends AbstractRecordWriter { + + private final Cluster cluster; + private final Session session; + private final StatementPlanner statementPlanner; + private final CqlSinkConfig config; + + /** + * Creates a new CQL-based record writer for testing. + * + * @param session the Cassandra session to use + * @param statementPlanner the statement planner for query preparation + * @param config the sink configuration + */ + @VisibleForTesting + CqlRecordWriter( + Session session, StatementPlanner statementPlanner, CqlSinkConfig config) { + Preconditions.checkArgument(session != null, "Session cannot be null"); + Preconditions.checkArgument(statementPlanner != null, "StatementPlanner cannot be null"); + Preconditions.checkArgument(config != null, "CqlSinkConfig cannot be null"); + + this.session = session; + this.statementPlanner = statementPlanner; + this.config = config; + this.cluster = null; + } + + /** + * Creates a new CQL-based record writer. + * + * @param builder the {@link ClusterBuilder} used to connect to Cassandra + * @param config the sink config that provides insert query and record format + * @throws RuntimeException if initialization fails + */ + public CqlRecordWriter(ClusterBuilder builder, CqlSinkConfig config) { + Preconditions.checkArgument(config != null, "CqlSinkConfig cannot be null"); + Preconditions.checkArgument(builder != null, "ClusterBuilder cannot be null"); + this.config = config; + + try { + this.cluster = builder.getCluster(); + } catch (Exception e) { + throw new RuntimeException( + "Failed to create Cassandra cluster from ClusterBuilder. " + + "Check your cluster configuration (contact points, credentials, etc.)", + e); + } + + try { + this.session = cluster.connect(); + } catch (Exception e) { + // Clean up cluster if session connection fails + if (cluster != null && !cluster.isClosed()) { + try { + cluster.close(); + } catch (Exception closeException) { + e.addSuppressed(closeException); + } + } + throw new RuntimeException( + "Failed to connect to Cassandra cluster. " + + "Check cluster availability and network connectivity", + e); + } + + this.statementPlanner = StatementPlannerFactory.create(config); + } + + /** + * Gets the Cassandra session used by this writer. + * + * @return the active session + */ + @Override + public Session getSession() { + return session; + } + + /** + * Extracts field values from the input record and creates a bound statement. + * + *

Delegates to the statement planner which orchestrates table resolution, column/value + * resolution, prepared statement caching, and value binding. + * + * @param input the structured record + * @return a {@link Statement} ready for async execution + */ + @Override + public Statement prepareStatement(T input) { + return statementPlanner.plan(input, session, config); + } + + /** Cleanup resources when the writer is closed. */ + @Override + public void close() { + try { + if (statementPlanner != null) { + statementPlanner.close(); + } + } finally { + super.close(); + } + } + + /** + * Gets the Cassandra cluster instance. + * + * @return the cluster used by this writer + */ + @Override + protected Cluster getCluster() { + return cluster; + } +} diff --git a/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/PojoRecordWriter.java b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/PojoRecordWriter.java new file mode 100644 index 0000000..19cf3c4 --- /dev/null +++ b/flink-connector-cassandra/src/main/java/org/apache/flink/connector/cassandra/sink/writer/PojoRecordWriter.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.writer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.cassandra.sink.config.PojoSinkConfig; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.util.Preconditions; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.datastax.driver.mapping.Mapper; +import com.datastax.driver.mapping.MappingManager; + +import java.util.Objects; + +/** + * A {@link CassandraRecordWriter} implementation that writes POJO records to Cassandra using the + * DataStax {@link Mapper}. + * + *

This writer: + * + *

    + *
  • Creating a Cassandra {@link Session} from a {@link ClusterBuilder} + *
  • Instantiating a {@link Mapper} using the provided POJO class + *
  • Executing insert/update statements using {@code mapper.saveQuery()} + *
  • Applying optional {@link com.datastax.driver.mapping.Mapper.Option}s from the config + *
+ * + * @param the type of the input POJO + */ +@Internal +public class PojoRecordWriter extends AbstractRecordWriter { + + private final Session session; + private final Cluster cluster; + private final Mapper mapper; + + /** + * Constructs a PojoRecordWriter using the provided builder and sink config. + * + * @param builder the cluster builder used to connect to Cassandra + * @param config the config containing POJO class, keyspace, and mapper options + * @throws RuntimeException if initialization fails + */ + public PojoRecordWriter(ClusterBuilder builder, PojoSinkConfig config) { + Preconditions.checkArgument(builder != null, "ClusterBuilder cannot be null"); + Preconditions.checkArgument(config != null, "PojoSinkConfig cannot be null"); + try { + this.cluster = builder.getCluster(); + this.session = cluster.connect(config.getKeyspace()); + + // Create mapper using MappingManager + MappingManager mappingManager = new MappingManager(session); + this.mapper = mappingManager.mapper(config.getPojoClass()); + + // Apply mapper options if provided + if (config.getMapperOptions() != null) { + Mapper.Option[] optionsArray = config.getMapperOptions().getMapperOptions(); + if (optionsArray != null && optionsArray.length > 0) { + mapper.setDefaultSaveOptions(optionsArray); + } + } + } catch (Exception e) { + throw new RuntimeException( + String.format( + "Failed to initialize PojoRecordWriter for keyspace '%s' and pojo %s", + config.getKeyspace(), config.getPojoClass().getName()), + e); + } + } + + /** + * Test-only constructor that accepts all dependencies directly. + * + *

This constructor is only for testing purposes and bypasses the normal initialization of + * the MappingManager. + * + * @param cluster the Cassandra cluster + * @param session the Cassandra session + * @param mapper the DataStax mapper for the POJO type + */ + @VisibleForTesting + PojoRecordWriter(Cluster cluster, Session session, Mapper mapper) { + this.cluster = Objects.requireNonNull(cluster, "cluster"); + this.session = Objects.requireNonNull(session, "session"); + this.mapper = Objects.requireNonNull(mapper, "mapper"); + } + + /** + * Gets the Cassandra session used by this writer. + * + * @return the active session + */ + @Override + public Session getSession() { + return session; + } + + /** + * Generates a CQL {@link Statement} for the given POJO using the DataStax mapper. + * + *

The mapper will generate an appropriate INSERT or UPDATE statement based on the POJO + * annotations and configured mapper options. + * + * @param input the POJO to serialize into a statement + * @return a statement ready for asynchronous execution + */ + @Override + public Statement prepareStatement(INPUT input) { + return mapper.saveQuery(input); + } + + /** + * Gets the Cassandra cluster instance. + * + * @return the cluster used by this writer + */ + @Override + protected Cluster getCluster() { + return cluster; + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/CassandraTestEnvironment.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/CassandraTestEnvironment.java index 7a8f5d3..356fcfe 100644 --- a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/CassandraTestEnvironment.java +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/CassandraTestEnvironment.java @@ -140,6 +140,10 @@ public void tearDown() throws Exception { stopEnv(); } + public String getHost() { + return cassandraContainer1.getHost(); + } + private void startEnv() throws Exception { // configure container start to wait until cassandra is ready to receive queries // start with retrials diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraCqlSinkITCase.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraCqlSinkITCase.java new file mode 100644 index 0000000..489ad3f --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraCqlSinkITCase.java @@ -0,0 +1,1820 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.connector.cassandra.CassandraTestEnvironment; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration; +import org.apache.flink.connector.cassandra.sink.planner.SinkPluggable; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Statement; +import com.datastax.driver.core.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.Update; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static com.datastax.driver.core.querybuilder.QueryBuilder.bindMarker; +import static com.datastax.driver.core.querybuilder.QueryBuilder.eq; +import static com.datastax.driver.core.querybuilder.QueryBuilder.ttl; +import static org.apache.flink.connector.cassandra.CassandraTestEnvironment.KEYSPACE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * Comprehensive integration tests for CassandraSink V2 with CQL queries covering Row and Tuple + * types in both static and dynamic modes. + */ +@Testcontainers +class CassandraCqlSinkITCase { + + @RegisterExtension + static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension(); + + // Static ClusterBuilder to avoid serialization issues + public static class TestClusterBuilder extends ClusterBuilder { + private final String host; + private final int port; + + public TestClusterBuilder(String host, int port) { + this.host = host; + this.port = port; + } + + @Override + protected Cluster buildCluster(Cluster.Builder builder) { + return builder.addContactPoint(host).withPort(port).build(); + } + } + + private static final CassandraTestEnvironment cassandraTestEnvironment = + new CassandraTestEnvironment(false); + + private static Integer port; + private static String host; + + // TypeInformation constants for Row-based tests + private static final TypeInformation ROW_S_S_I = + Types.ROW(Types.STRING, Types.STRING, Types.INT); + private static final TypeInformation ROW_S_S = Types.ROW(Types.STRING, Types.STRING); + private static final TypeInformation ROW_I_S = Types.ROW(Types.INT, Types.STRING); + private static final TypeInformation ROW_I_S_S = + Types.ROW(Types.INT, Types.STRING, Types.STRING); + private static final TypeInformation ROW_S_I_S = + Types.ROW(Types.STRING, Types.INT, Types.STRING); + private static final TypeInformation ROW_S = Types.ROW(Types.STRING); + + @BeforeAll + static void setupCassandra() throws Exception { + cassandraTestEnvironment.startUp(); + createTestTables(); + port = cassandraTestEnvironment.getPort(); + host = cassandraTestEnvironment.getHost(); + } + + @AfterAll + static void tearDownCassandra() throws Exception { + dropTestTables(); + cassandraTestEnvironment.tearDown(); + } + + @BeforeEach + void clearTables() { + // Clear all test tables + clearTable("test_table"); + clearTable("test_table_composite"); + clearTable("test_table_lwt"); + clearTable("test_table_ttl"); + clearTable("test_table_even"); + clearTable("test_table_odd"); + clearTable("\"UserEvents\""); + } + + private void clearTable(String tableName) { + try { + String truncateQuery = String.format("TRUNCATE %s.%s;", KEYSPACE, tableName); + cassandraTestEnvironment.executeRequestWithTimeout(truncateQuery); + } catch (Exception ignored) { + // Ignore if table doesn't exist + } + } + + private static void createTestTables() { + // Simple table for basic tests + String createSimple = + String.format( + "CREATE TABLE IF NOT EXISTS %s.test_table (" + + "id text PRIMARY KEY, " + + "name text, " + + "age int" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createSimple); + + // Table with composite key + String createComposite = + String.format( + "CREATE TABLE IF NOT EXISTS %s.test_table_composite (" + + "pk text, " + + "ck int, " + + "value text, " + + "PRIMARY KEY (pk, ck)" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createComposite); + + // Table for LWT tests + String createLWT = + String.format( + "CREATE TABLE IF NOT EXISTS %s.test_table_lwt (" + + "id text PRIMARY KEY, " + + "name text, " + + "version int" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createLWT); + + // Table for TTL tests + String createTTL = + String.format( + "CREATE TABLE IF NOT EXISTS %s.test_table_ttl (" + + "id text PRIMARY KEY, " + + "value text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createTTL); + + // Tables for routing tests + String createEven = + String.format( + "CREATE TABLE IF NOT EXISTS %s.test_table_even (" + + "id int PRIMARY KEY, " + + "data text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createEven); + + String createOdd = + String.format( + "CREATE TABLE IF NOT EXISTS %s.test_table_odd (" + + "id int PRIMARY KEY, " + + "data text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createOdd); + + // Table for quoted identifier tests - using case-sensitive identifiers + String createQuoted = + String.format( + "CREATE TABLE IF NOT EXISTS %s.\"UserEvents\" (" + + "\"UserId\" text PRIMARY KEY, " + + "\"EventName\" text, " + + "\"Value\" int" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createQuoted); + } + + private static void dropTestTables() { + String[] tables = { + "test_table", "test_table_composite", "test_table_lwt", + "test_table_ttl", "test_table_even", "test_table_odd", + "\"UserEvents\"" + }; + + for (String table : tables) { + try { + String dropQuery = String.format("DROP TABLE IF EXISTS %s.%s;", KEYSPACE, table); + cassandraTestEnvironment.executeRequestWithTimeout(dropQuery); + } catch (Exception ignored) { + // Ignore errors + } + } + } + + @Test + void testStaticInsertBasicRow() throws Exception { + // Tests basic INSERT operation with Row type in static mode + List rows = new ArrayList<>(); + rows.add(Row.of("row1", "Alice", 25)); + rows.add(Row.of("row2", "Bob", 30)); + rows.add(Row.of("row3", "Charlie", 35)); + + String insertQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES (?, ?, ?)", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + + TypeInformation rowTypeInfo = Types.ROW(Types.STRING, Types.STRING, Types.INT); + executeSinkAndWait(rows, config, rowTypeInfo); + + // Verify data was inserted + verifyRowExists("test_table", "id", "row1", "name", "Alice", "age", 25); + verifyRowExists("test_table", "id", "row2", "name", "Bob", "age", 30); + verifyRowExists("test_table", "id", "row3", "name", "Charlie", "age", 35); + } + + @Test + void testStaticInsertWithCaseSensitiveIdentifiersRow() throws Exception { + // Tests INSERT with case-sensitive quoted table and column identifiers + // Test with case-sensitive column names (quoted in table creation) + List rows = Collections.singletonList(Row.of("q1", "quoted-test", 42)); + + String insertQuery = + String.format( + "INSERT INTO %s.\"UserEvents\" (\"UserId\", \"EventName\", \"Value\") VALUES (?, ?, ?)", + KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + + executeSinkAndWait(rows, config, ROW_S_S_I); + + // Verify with quoted columns + String selectQuery = + String.format( + "SELECT \"UserId\", \"EventName\", \"Value\" FROM %s.\"UserEvents\" WHERE \"UserId\" = 'q1'", + KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row).isNotNull(); + assertThat(row.getString("UserId")).isEqualTo("q1"); + assertThat(row.getString("EventName")).isEqualTo("quoted-test"); + assertThat(row.getInt("Value")).isEqualTo(42); + } + + @Test + void testStaticInsertWithTTLRow() throws Exception { + // Tests INSERT with static TTL clause that expires records after specified seconds + List rows = Collections.singletonList(Row.of("ttl1", "will-expire")); + + String insertQuery = + String.format( + "INSERT INTO %s.test_table_ttl (id, value) VALUES (?, ?) USING TTL 5", + KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + + executeSinkAndWait(rows, config, ROW_S_S); + + // Give some time for write to complete + Thread.sleep(500); + + // Verify row exists initially + verifyRowExists("test_table_ttl", "id", "ttl1", "value", "will-expire"); + + // Wait for TTL to expire + Thread.sleep(6000); + + // Verify row is gone + String selectQuery = + String.format("SELECT * FROM %s.test_table_ttl WHERE id = 'ttl1'", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + assertThat(rs.one()).isNull(); + } + + @Test + void testStaticInsertWithTimestampRow() throws Exception { + // Tests INSERT with custom TIMESTAMP clause for setting write time + List rows = Collections.singletonList(Row.of("ts1", "with-timestamp", 100)); + + long timestamp = System.currentTimeMillis() * 1000; // microseconds + String insertQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES (?, ?, ?) USING TIMESTAMP %d", + KEYSPACE, timestamp); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + + executeSinkAndWait(rows, config, ROW_S_S_I); + + // Verify writetime + String selectQuery = + String.format( + "SELECT writetime(name) FROM %s.test_table WHERE id = 'ts1'", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row.getLong(0)).isEqualTo(timestamp); + } + + @Test + void testStaticInsertIfNotExistsRow() throws Exception { + // Tests INSERT with IF NOT EXISTS clause for conditional inserts + // First insert should succeed + List rows1 = Collections.singletonList(Row.of("lwt1", "first", 1)); + + String insertQuery = + String.format( + "INSERT INTO %s.test_table_lwt (id, name, version) VALUES (?, ?, ?) IF NOT EXISTS", + KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + + executeSinkAndWait(rows1, config, ROW_S_S_I); + + verifyRowExists("test_table_lwt", "id", "lwt1", "name", "first", "version", 1); + + // Second insert with same key should be ignored + List rows2 = Collections.singletonList(Row.of("lwt1", "second", 2)); + executeSinkAndWait(rows2, config, ROW_S_S_I); + + // Verify original data remains + verifyRowExists("test_table_lwt", "id", "lwt1", "name", "first", "version", 1); + } + + @Test + void testStaticInsertWithIgnoreNullFieldsRow() throws Exception { + // Tests INSERT with ignoreNullFields=true to prevent tombstone creation + // Define TypeInformation for Row to avoid Kryo serialization + TypeInformation rowTypeInfo = Types.ROW(Types.STRING, Types.STRING, Types.INT); + + // First insert complete row + List rows1 = Collections.singletonList(Row.of("null1", "initial", 50)); + + String insertQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES (?, ?, ?)", KEYSPACE); + + CqlSinkConfig config = + CqlSinkConfig.forRow().withQuery(insertQuery).withIgnoreNullFields(true); + + executeSinkAndWait(rows1, config, rowTypeInfo); + + // Insert with null field - should not create tombstone + List rows2 = Collections.singletonList(Row.of("null1", null, 60)); + executeSinkAndWait(rows2, config, rowTypeInfo); + + // Verify name is still present (not tombstoned) + verifyRowExists("test_table", "id", "null1", "name", "initial", "age", 60); + } + + // ============= Static INSERT Tests for Tuple ============= + + @Test + void testStaticInsertBasicTuple() throws Exception { + // Tests basic INSERT operation with Tuple type in static mode + List> tuples = + Arrays.asList(Tuple3.of("tuple1", "Alice", 25), Tuple3.of("tuple2", "Bob", 30)); + + String insertQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES (?, ?, ?)", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forTuple().withQuery(insertQuery); + + executeSinkAndWait((List) (List) tuples, config); + + verifyRowExists("test_table", "id", "tuple1", "name", "Alice", "age", 25); + verifyRowExists("test_table", "id", "tuple2", "name", "Bob", "age", 30); + } + + @Test + void testStaticInsertWithTTLTuple() throws Exception { + // Tests INSERT with TTL using Tuple type that expires after specified seconds + List> tuples = + Collections.singletonList(Tuple2.of("ttl_tuple", "expires")); + + String insertQuery = + String.format( + "INSERT INTO %s.test_table_ttl (id, value) VALUES (?, ?) USING TTL 5", + KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forTuple().withQuery(insertQuery); + + executeSinkAndWait((List) (List) tuples, config); + + // Give some time for write to complete + Thread.sleep(500); + + verifyRowExists("test_table_ttl", "id", "ttl_tuple", "value", "expires"); + + Thread.sleep(6000); + + String selectQuery = + String.format("SELECT * FROM %s.test_table_ttl WHERE id = 'ttl_tuple'", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + assertThat(rs.one()).isNull(); + } + + // ============= Static UPDATE Tests for Row ============= + + @Test + void testStaticUpdateBasicRow() throws Exception { + // Tests basic UPDATE operation with Row type in static mode + // First insert some data + String setupQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES ('update1', 'Old', 20)", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(setupQuery); + + // Update the data + List rows = Collections.singletonList(Row.of("New", 25, "update1")); + + String updateQuery = + String.format("UPDATE %s.test_table SET name = ?, age = ? WHERE id = ?", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(updateQuery); + + executeSinkAndWait(rows, config, ROW_S_I_S); + + verifyRowExists("test_table", "id", "update1", "name", "New", "age", 25); + } + + @Test + void testStaticUpdateMultipleColumnsRow() throws Exception { + // Tests UPDATE operations targeting different columns separately + // Setup + String setupQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES ('multi1', 'Initial', 30)", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(setupQuery); + + // Update only name + List rows1 = Arrays.asList(Row.of("Updated", "multi1")); + String updateQuery1 = + String.format("UPDATE %s.test_table SET name = ? WHERE id = ?", KEYSPACE); + CqlSinkConfig config1 = CqlSinkConfig.forRow().withQuery(updateQuery1); + executeSinkAndWait(rows1, config1, ROW_S_S); + + // Update only age + List rows2 = Arrays.asList(Row.of(35, "multi1")); + String updateQuery2 = + String.format("UPDATE %s.test_table SET age = ? WHERE id = ?", KEYSPACE); + CqlSinkConfig config2 = CqlSinkConfig.forRow().withQuery(updateQuery2); + executeSinkAndWait(rows2, config2, ROW_I_S); + + verifyRowExists("test_table", "id", "multi1", "name", "Updated", "age", 35); + } + + @Test + void testStaticUpdateWithCompositeKeyRow() throws Exception { + // Tests UPDATE with composite primary key (partition key + clustering key) + // Setup + String setupQuery = + String.format( + "INSERT INTO %s.test_table_composite (pk, ck, value) VALUES ('pk1', 1, 'old')", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(setupQuery); + + // Update + List rows = Arrays.asList(Row.of("new", "pk1", 1)); + String updateQuery = + String.format( + "UPDATE %s.test_table_composite SET value = ? WHERE pk = ? AND ck = ?", + KEYSPACE); + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(updateQuery); + executeSinkAndWait(rows, config, ROW_S_S_I); + + String selectQuery = + String.format( + "SELECT * FROM %s.test_table_composite WHERE pk = 'pk1' AND ck = 1", + KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row.getString("value")).isEqualTo("new"); + } + + @Test + void testStaticUpdateWithTTLRow() throws Exception { + // Tests UPDATE with TTL clause that expires updated columns after specified seconds + // Setup + String setupQuery = + String.format( + "INSERT INTO %s.test_table_ttl (id, value) VALUES ('ttl_update', 'initial')", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(setupQuery); + + // Update with TTL + List rows = Arrays.asList(Row.of("updated", "ttl_update")); + String updateQuery = + String.format( + "UPDATE %s.test_table_ttl USING TTL 5 SET value = ? WHERE id = ?", + KEYSPACE); + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(updateQuery); + executeSinkAndWait(rows, config, ROW_S_S); + + // Give some time for write to complete + Thread.sleep(500); + + verifyRowExists("test_table_ttl", "id", "ttl_update", "value", "updated"); + + Thread.sleep(6000); + + // The value column should expire but row remains + String selectQuery = + String.format( + "SELECT value FROM %s.test_table_ttl WHERE id = 'ttl_update'", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row.isNull("value")).isTrue(); + } + + @Test + void testStaticUpdateBasicTuple() throws Exception { + // Tests basic UPDATE operation with Tuple type in static mode + // Setup + String setupQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES ('tuple_up', 'Old', 20)", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(setupQuery); + + // Update + List> tuples = + Collections.singletonList(Tuple3.of("New", 25, "tuple_up")); + + String updateQuery = + String.format("UPDATE %s.test_table SET name = ?, age = ? WHERE id = ?", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forTuple().withQuery(updateQuery); + + executeSinkAndWait((List) (List) tuples, config); + + verifyRowExists("test_table", "id", "tuple_up", "name", "New", "age", 25); + } + + private static class EvenOddTableResolver implements TableResolver { + @Override + public TableRef resolve(Row record) { + int id = (int) record.getField(0); + String tableName = (id % 2 == 0) ? "test_table_even" : "test_table_odd"; + return new TableRef(KEYSPACE, tableName); + } + } + + private static class InsertColumnResolver implements ColumnValueResolver { + @Override + public Kind kind() { + return Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + int id = (int) record.getField(0); + String data = (String) record.getField(1); + return ResolvedWrite.insert(Arrays.asList("id", "data"), new Object[] {id, data}); + } + } + + @Test + void testDynamicInsertWithTableRoutingRow() throws Exception { + // Tests dynamic mode with TableResolver routing records to different tables based on + // content + // Create dynamic pluggable that routes to different tables based on id + TableResolver tableResolver = new EvenOddTableResolver(); + ColumnValueResolver columnResolver = new InsertColumnResolver(); + + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(tableResolver) + .withColumnValueResolver(columnResolver) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + List rows = + Arrays.asList( + Row.of(1, "odd-one"), + Row.of(2, "even-two"), + Row.of(3, "odd-three"), + Row.of(4, "even-four")); + + executeSinkAndWait(rows, config, ROW_I_S); + + // Verify routing + verifyRowExists("test_table_odd", "id", 1, "data", "odd-one"); + verifyRowExists("test_table_even", "id", 2, "data", "even-two"); + verifyRowExists("test_table_odd", "id", 3, "data", "odd-three"); + verifyRowExists("test_table_even", "id", 4, "data", "even-four"); + } + + private static class CompositeTableResolver implements TableResolver { + @Override + public TableRef resolve(Row record) { + return new TableRef(KEYSPACE, "test_table_composite"); + } + } + + private static class UpdateColumnResolver implements ColumnValueResolver { + @Override + public Kind kind() { + return Kind.UPDATE; + } + + @Override + public ResolvedWrite resolve(Row record) { + String pk = (String) record.getField(0); + int ck = (int) record.getField(1); + String value = (String) record.getField(2); + return ResolvedWrite.update( + Collections.singletonList("value"), + new Object[] {value}, + Arrays.asList("pk", "ck"), + new Object[] {pk, ck}); + } + } + + @Test + void testDynamicUpdateWithCompositeKeysRow() throws Exception { + // Tests dynamic mode UPDATE with composite keys using ColumnValueResolver + // Setup data + String setupQuery1 = + String.format( + "INSERT INTO %s.test_table_composite (pk, ck, value) VALUES ('pk1', 1, 'old1')", + KEYSPACE); + String setupQuery2 = + String.format( + "INSERT INTO %s.test_table_composite (pk, ck, value) VALUES ('pk1', 2, 'old2')", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(setupQuery1); + cassandraTestEnvironment.executeRequestWithTimeout(setupQuery2); + + TableResolver tableResolver = new CompositeTableResolver(); + ColumnValueResolver columnResolver = new UpdateColumnResolver(); + + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(tableResolver) + .withColumnValueResolver(columnResolver) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + List rows = Arrays.asList(Row.of("pk1", 1, "new1"), Row.of("pk1", 2, "new2")); + + executeSinkAndWait(rows, config, ROW_S_I_S); + + // Verify updates + String selectQuery1 = + String.format( + "SELECT * FROM %s.test_table_composite WHERE pk = 'pk1' AND ck = 1", + KEYSPACE); + ResultSet rs1 = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery1); + assertThat(rs1.one().getString("value")).isEqualTo("new1"); + + String selectQuery2 = + String.format( + "SELECT * FROM %s.test_table_composite WHERE pk = 'pk1' AND ck = 2", + KEYSPACE); + ResultSet rs2 = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery2); + assertThat(rs2.one().getString("value")).isEqualTo("new2"); + } + + private static class SimpleTableResolver implements TableResolver { + @Override + public TableRef resolve(Row record) { + return new TableRef(KEYSPACE, "test_table"); + } + } + + private static class SimpleInsertColumnResolver implements ColumnValueResolver { + @Override + public Kind kind() { + return Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + String id = (String) record.getField(0); + String name = (String) record.getField(1); + int age = (int) record.getField(2); + return ResolvedWrite.insert( + Arrays.asList("id", "name", "age"), new Object[] {id, name, age}); + } + } + + @Test + void testDynamicWithTableAndColumnsRow() throws Exception { + // Tests basic dynamic mode with TableResolver and ColumnValueResolver without + // CqlClauseResolver + // Simple test for dynamic mode without CqlClauseResolver + TableResolver tableResolver = new SimpleTableResolver(); + ColumnValueResolver columnResolver = new SimpleInsertColumnResolver(); + + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(tableResolver) + .withColumnValueResolver(columnResolver) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + List rows = Arrays.asList(Row.of("dyn1", "dynamic", 100)); + + executeSinkAndWait(rows, config, ROW_S_S_I); + + verifyRowExists("test_table", "id", "dyn1", "name", "dynamic", "age", 100); + } + + // ============= Cross-cutting Tests ============= + + @Test + void testBackpressureWithRequestConfigurationRow() throws Exception { + // Tests backpressure handling with limited concurrent requests and large batch size + // Test with limited concurrency to simulate backpressure + RequestConfiguration requestConfig = + RequestConfiguration.builder().setMaxConcurrentRequests(1).build(); + + List rows = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + rows.add(Row.of("bp" + i, "name" + i, i)); + } + + String insertQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES (?, ?, ?)", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + + executeSinkAndWaitWithRequestConfig(rows, config, requestConfig, ROW_S_S_I); + + // Verify all records were inserted despite backpressure + for (int i = 0; i < 100; i++) { + verifyRowExists("test_table", "id", "bp" + i, "name", "name" + i, "age", i); + } + } + + @Test + void testFatalErrorPropagationRow() throws Exception { + // Tests proper error propagation when sink configuration is invalid (wrong parameter count) + List rows = Collections.singletonList(Row.of("error1", "test", 25)); + + // Invalid query with wrong number of parameters + String invalidQuery = + String.format("INSERT INTO %s.test_table (id, name, age) VALUES (?, ?)", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(invalidQuery); + + assertThatThrownBy(() -> executeSinkAndWait(rows, config, ROW_S_S_I)) + .isInstanceOf(org.apache.flink.runtime.client.JobExecutionException.class) + .hasStackTraceContaining("Column count") + .hasStackTraceContaining("must match value placeholder count"); + } + + @Test + void testLargeBatchRow() throws Exception { + // Tests processing large batches (1000 records) for performance and stability + List rows = new ArrayList<>(); + for (int i = 0; i < 1000; i++) { + rows.add(Row.of("batch" + i, "name" + i, i % 100)); + } + + String insertQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES (?, ?, ?)", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + + executeSinkAndWait(rows, config, ROW_S_S_I); + + // Spot check some records + verifyRowExists("test_table", "id", "batch0", "name", "name0", "age", 0); + verifyRowExists("test_table", "id", "batch500", "name", "name500", "age", 0); + verifyRowExists("test_table", "id", "batch999", "name", "name999", "age", 99); + } + + // ============= Static Mode Requirement Tests ============= + + @Test + void testStaticModeRequiresFQN() throws Exception { + // Tests that static mode requires fully qualified keyspace.table names + List rows = Collections.singletonList(Row.of("fqn1", "test", 25)); + + // Query without keyspace - should fail + String invalidQuery = "INSERT INTO test_table (id, name, age) VALUES (?, ?, ?)"; + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(invalidQuery); + + assertThatThrownBy(() -> executeSinkAndWait(rows, config, ROW_S_S_I)) + .isInstanceOf(JobExecutionException.class) + .hasStackTraceContaining("Static mode requires fully qualified keyspace.table"); + } + + @Test + void testBindMarkerParameterCountValidation() throws Exception { + // Tests bind marker parameter count validation and error handling + // Test parameter count validation and error handling with bind markers + + // Setup - insert some test data + for (int i = 1; i <= 3; i++) { + String query = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES ('in%d', 'name%d', %d)", + KEYSPACE, i, i, i * 10); + cassandraTestEnvironment.executeRequestWithTimeout(query); + } + + // Test simple UPDATE without WHERE IN first + List simpleRows = Collections.singletonList(Row.of("updated", "in1")); + String simpleQuery = + String.format("UPDATE %s.test_table SET name = ? WHERE id = ?", KEYSPACE); + CqlSinkConfig simpleConfig = CqlSinkConfig.forRow().withQuery(simpleQuery); + executeSinkAndWait(simpleRows, simpleConfig, ROW_S_S); + + // Verify simple update worked + verifyRowExists("test_table", "id", "in1", "name", "updated"); + + // Test failure case - wrong number of values + List wrongRows = Collections.singletonList(Row.of("failed")); + + assertThatThrownBy(() -> executeSinkAndWait(wrongRows, simpleConfig, ROW_S)) + .isInstanceOf(JobExecutionException.class) + .hasStackTraceContaining("UPDATE parameter count mismatch"); + } + + @Test + void testWhereInClauseWithBindMarkers() throws Exception { + // Tests WHERE IN clause with bind markers mixed with literal values + // Test WHERE IN clause with bind markers (static mode requires at least one ?) + + // Setup - insert test data + for (int i = 1; i <= 5; i++) { + String query = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES ('batch%d', 'name%d', %d)", + KEYSPACE, i, i, i * 10); + cassandraTestEnvironment.executeRequestWithTimeout(query); + } + + // Test UPDATE with WHERE id IN (mix of literal and bind marker) + // Note: Static mode requires at least one bind marker in WHERE clause + List rows = new ArrayList<>(); + rows.add(Row.of("batch_updated", "batch1")); + + String updateQuery = + String.format( + "UPDATE %s.test_table SET name = ? WHERE id IN (?, 'batch3')", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(updateQuery); + TypeInformation rowTypeInfo = Types.ROW(Types.STRING, Types.STRING); + executeSinkAndWait(rows, config, rowTypeInfo); + + // Verify updates applied to specified records + verifyRowExists("test_table", "id", "batch1", "name", "batch_updated"); + verifyRowExists("test_table", "id", "batch3", "name", "batch_updated"); + + // Verify other records unchanged + verifyRowExists("test_table", "id", "batch2", "name", "name2"); + verifyRowExists("test_table", "id", "batch4", "name", "name4"); + verifyRowExists("test_table", "id", "batch5", "name", "name5"); + } + + @Test + void testSetColumnsOrdering() throws Exception { + // Tests UPDATE/INSERT with column ordering different from Row field order + // Setup - insert initial data + String setupQuery = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES ('order1', 'initial', 10)", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(setupQuery); + + // Test UPDATE with column order different from record order + // Query expects: age, name, id + // Row provides: age, name, id (matching the query order) + List rows = Collections.singletonList(Row.of(30, "reordered", "order1")); + + String updateQuery = + String.format("UPDATE %s.test_table SET age = ?, name = ? WHERE id = ?", KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(updateQuery); + executeSinkAndWait(rows, config, ROW_I_S_S); + + // Verify the update with correct column mapping + verifyRowExists("test_table", "id", "order1", "name", "reordered", "age", 30); + + // Test INSERT with different column ordering + List insertRows = Collections.singletonList(Row.of(40, "order2", "inserted")); + + String insertQuery = + String.format( + "INSERT INTO %s.test_table (age, id, name) VALUES (?, ?, ?)", KEYSPACE); + + CqlSinkConfig insertConfig = CqlSinkConfig.forRow().withQuery(insertQuery); + executeSinkAndWait(insertRows, insertConfig, ROW_I_S_S); + + // Verify insert worked with correct mapping + verifyRowExists("test_table", "id", "order2", "name", "inserted", "age", 40); + } + + @Test + void testQuotedIdentifiersInsertAndUpdate() throws Exception { + // Tests INSERT and UPDATE with case-sensitive quoted column identifiers + // Test INSERT with case-sensitive quoted identifiers + List insertRows = Collections.singletonList(Row.of("user123", "login", 42)); + + String insertQuery = + String.format( + "INSERT INTO %s.\"UserEvents\" (\"UserId\", \"EventName\", \"Value\") VALUES (?, ?, ?)", + KEYSPACE); + + CqlSinkConfig insertConfig = CqlSinkConfig.forRow().withQuery(insertQuery); + executeSinkAndWait(insertRows, insertConfig, ROW_S_S_I); + + // Verify insert with quoted columns - note case sensitivity + String selectQuery = + String.format( + "SELECT \"UserId\", \"EventName\", \"Value\" FROM %s.\"UserEvents\" WHERE \"UserId\" = 'user123'", + KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row.getString("UserId")).isEqualTo("user123"); + assertThat(row.getString("EventName")).isEqualTo("login"); + assertThat(row.getInt("Value")).isEqualTo(42); + + // Test UPDATE with quoted identifiers + List updateRows = Collections.singletonList(Row.of("logout", 100, "user123")); + + String updateQuery = + String.format( + "UPDATE %s.\"UserEvents\" SET \"EventName\" = ?, \"Value\" = ? WHERE \"UserId\" = ?", + KEYSPACE); + + CqlSinkConfig updateConfig = CqlSinkConfig.forRow().withQuery(updateQuery); + executeSinkAndWait(updateRows, updateConfig, ROW_S_I_S); + + // Verify update + rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + row = rs.one(); + assertThat(row.getString("EventName")).isEqualTo("logout"); + assertThat(row.getInt("Value")).isEqualTo(100); + } + + private void executeSinkAndWait(List records, CqlSinkConfig config) throws Exception { + executeSinkAndWait(records, config, null); + } + + private void executeSinkAndWait( + List records, CqlSinkConfig config, TypeInformation typeInfo) + throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream stream; + if (typeInfo != null) { + stream = env.fromCollection(records, typeInfo); + } else { + stream = env.fromCollection(records); + } + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + stream.sinkTo(sink); + env.execute(); + } + + private void executeSinkAndWaitWithRequestConfig( + List records, CqlSinkConfig config, RequestConfiguration requestConfig) + throws Exception { + executeSinkAndWaitWithRequestConfig(records, config, requestConfig, null); + } + + private void executeSinkAndWaitWithRequestConfig( + List records, + CqlSinkConfig config, + RequestConfiguration requestConfig, + TypeInformation typeInfo) + throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream stream; + if (typeInfo != null) { + stream = env.fromCollection(records, typeInfo); + } else { + stream = env.fromCollection(records); + } + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .setRequestConfiguration(requestConfig) + .build(); + + stream.sinkTo(sink); + env.execute(); + } + + private void verifyRowExists(String table, Object... columnValuePairs) { + assertThat(columnValuePairs.length % 2).isEqualTo(0); + + StringBuilder whereClause = new StringBuilder(); + for (int i = 0; i < columnValuePairs.length; i += 2) { + String column = (String) columnValuePairs[i]; + if (column.endsWith("id") || column.endsWith("pk") || column.endsWith("ck")) { + if (whereClause.length() > 0) { + whereClause.append(" AND "); + } + Object value = columnValuePairs[i + 1]; + if (value instanceof String) { + whereClause.append(column).append(" = '").append(value).append("'"); + } else { + whereClause.append(column).append(" = ").append(value); + } + } + } + + String selectQuery = + String.format("SELECT * FROM %s.%s WHERE %s", KEYSPACE, table, whereClause); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row).isNotNull(); + + for (int i = 0; i < columnValuePairs.length; i += 2) { + String column = (String) columnValuePairs[i]; + Object expectedValue = columnValuePairs[i + 1]; + + if (expectedValue instanceof String) { + assertThat(row.getString(column)).isEqualTo(expectedValue); + } else if (expectedValue instanceof Integer) { + assertThat(row.getInt(column)).isEqualTo(expectedValue); + } + } + } + + @Test + void testUsingAndIfTogether() throws Exception { + // Tests combining USING clauses (TTL/TIMESTAMP) with IF clauses in static mode + // Create a table with version column for LWT + String createTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.versioned_table (" + + "id text PRIMARY KEY, " + + "value text, " + + "version int" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createTable); + + // First insert a row with initial version + String initialInsert = + String.format( + "INSERT INTO %s.versioned_table (id, value, version) VALUES ('v1', 'initial', 1)", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(initialInsert); + + // Test UPDATE with USING TTL (static) and IF condition together + // Static mode only supports literal USING/IF clauses + List updateRows = Collections.singletonList(Row.of("updated", "v1")); + + String updateQuery = + String.format( + "UPDATE %s.versioned_table USING TTL 5 SET value = ? WHERE id = ? IF version = 1", + KEYSPACE); + + CqlSinkConfig updateConfig = CqlSinkConfig.forRow().withQuery(updateQuery); + executeSinkAndWait(updateRows, updateConfig, ROW_S_S); + + // Verify the update and TTL + Thread.sleep(1000); // Wait a bit for TTL to be visible + + String selectQuery = + String.format( + "SELECT id, value, version, TTL(value) as ttl_value FROM %s.versioned_table WHERE id = 'v1'", + KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row).isNotNull(); + assertThat(row.getString("value")).isEqualTo("updated"); + int ttl = row.getInt("ttl_value"); + assertThat(ttl).isLessThanOrEqualTo(5).isGreaterThan(0); + + // Test static INSERT with USING TTL and TIMESTAMP together (static values) + List insertRows = Collections.singletonList(Row.of("v2", "value2")); + long timestamp = System.currentTimeMillis() * 1000; // microseconds + + String insertQuery = + String.format( + "INSERT INTO %s.versioned_table (id, value) VALUES (?, ?) USING TTL 5 AND TIMESTAMP %d", + KEYSPACE, timestamp); + + CqlSinkConfig insertConfig = CqlSinkConfig.forRow().withQuery(insertQuery); + executeSinkAndWait(insertRows, insertConfig, ROW_S_S); + + // Verify the insert with TTL and TIMESTAMP + String verifyQuery = + String.format( + "SELECT id, value, TTL(value) as ttl_value, WRITETIME(value) as write_time FROM %s.versioned_table WHERE id = 'v2'", + KEYSPACE); + ResultSet rs2 = cassandraTestEnvironment.executeRequestWithTimeout(verifyQuery); + com.datastax.driver.core.Row row2 = rs2.one(); + assertThat(row2).isNotNull(); + assertThat(row2.getString("value")).isEqualTo("value2"); + + // TTL should be around 5 seconds + int ttl2 = row2.getInt("ttl_value"); + assertThat(ttl2).isLessThanOrEqualTo(5).isGreaterThan(0); + + // Timestamp should match what we set + assertThat(row2.getLong("write_time")).isEqualTo(timestamp); + } + + // Static inner classes to avoid serialization issues with anonymous classes + private static class DynamicTableResolver implements TableResolver { + private final String keyspace; + + public DynamicTableResolver(String keyspace) { + this.keyspace = keyspace; + } + + @Override + public TableRef resolve(Row record) { + int id = (int) record.getField(0); + String tableName = (id % 2 == 0) ? "dynamic_even" : "dynamic_odd"; + return new TableRef(keyspace, tableName); + } + } + + private static class DynamicColumnResolver implements ColumnValueResolver { + @Override + public ColumnValueResolver.Kind kind() { + return ColumnValueResolver.Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + return ResolvedWrite.insert( + Arrays.asList("id", "name", "category"), + new Object[] {record.getField(0), record.getField(1), record.getField(2)}); + } + } + + private static class SelectiveColumnResolver implements ColumnValueResolver { + @Override + public ColumnValueResolver.Kind kind() { + return ColumnValueResolver.Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + int id = (int) record.getField(0); + String requiredField = (String) record.getField(1); + boolean includeOptional = (boolean) record.getField(2); + + List columns = new ArrayList<>(); + List values = new ArrayList<>(); + + // Always include required fields + columns.add("id"); + values.add(id); + columns.add("required_field"); + values.add(requiredField); + + // Conditionally include optional field + if (includeOptional) { + columns.add("optional_field"); + values.add("optional_" + id); + } + + // Always add computed field + columns.add("computed_field"); + values.add("computed_" + id); + + return ResolvedWrite.insert(columns, values.toArray()); + } + } + + private static class FixedTableResolver implements TableResolver { + private final String keyspace; + private final String table; + + public FixedTableResolver(String keyspace, String table) { + this.keyspace = keyspace; + this.table = table; + } + + @Override + public TableRef resolve(Row record) { + return new TableRef(keyspace, table); + } + } + + private static class InsertAllColumnsResolver implements ColumnValueResolver { + @Override + public ColumnValueResolver.Kind kind() { + return ColumnValueResolver.Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + return ResolvedWrite.insert( + Arrays.asList("id", "data", "priority"), + new Object[] {record.getField(0), record.getField(1), record.getField(2)}); + } + } + + private static class ConsistencyLevelCustomizer implements StatementCustomizer { + @Override + public void apply(Statement statement, Row record) { + // Set consistency level based on priority field + String priority = (String) record.getField(2); + if ("high".equals(priority)) { + statement.setConsistencyLevel(ConsistencyLevel.QUORUM); + } else { + statement.setConsistencyLevel(ConsistencyLevel.ONE); + } + // Mark all statements as idempotent + statement.setIdempotent(true); + } + } + + @Test + void testIgnoreNullOnUpdate() throws Exception { + // Tests ignoreNullFields behavior on UPDATE to prevent tombstone creation + // Insert initial data + String initialInsert = + String.format( + "INSERT INTO %s.test_table (id, name, age) VALUES ('nulltest1', 'John', 30)", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(initialInsert); + + // Test UPDATE with null values and ignoreNullFields enabled + // The null value for age should be ignored + List updateRows = Collections.singletonList(Row.of("Jane", null, "nulltest1")); + + String updateQuery = + String.format("UPDATE %s.test_table SET name = ?, age = ? WHERE id = ?", KEYSPACE); + + CqlSinkConfig updateConfig = + CqlSinkConfig.forRow().withQuery(updateQuery).withIgnoreNullFields(true); + + TypeInformation rowTypeInfo = Types.ROW(Types.STRING, Types.INT, Types.STRING); + executeSinkAndWait(updateRows, updateConfig, rowTypeInfo); + + // Verify that name was updated but age retained its original value + String selectQuery = + String.format( + "SELECT id, name, age FROM %s.test_table WHERE id = 'nulltest1'", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row).isNotNull(); + assertThat(row.getString("name")).isEqualTo("Jane"); // Updated + assertThat(row.getInt("age")).isEqualTo(30); // Retained original value + + // Test UPDATE with all null values except WHERE clause + List allNullUpdateRows = Collections.singletonList(Row.of(null, null, "nulltest1")); + + executeSinkAndWait(allNullUpdateRows, updateConfig, rowTypeInfo); + + // Verify nothing was changed + rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + row = rs.one(); + assertThat(row).isNotNull(); + assertThat(row.getString("name")).isEqualTo("Jane"); // Still Jane + assertThat(row.getInt("age")).isEqualTo(30); // Still 30 + + // Test UPDATE without ignoreNullFields (default behavior) + List updateWithNullRows = Collections.singletonList(Row.of("Bob", null, "nulltest1")); + + CqlSinkConfig noIgnoreConfig = + CqlSinkConfig.forRow().withQuery(updateQuery).withIgnoreNullFields(false); + + executeSinkAndWait(updateWithNullRows, noIgnoreConfig, rowTypeInfo); + + // Verify that null was actually set (age should be null now) + rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + row = rs.one(); + assertThat(row).isNotNull(); + assertThat(row.getString("name")).isEqualTo("Bob"); // Updated + assertThat(row.isNull("age")).isTrue(); // Set to null + } + + @Test + void testBadQueryTypesFailFast() { + // Tests that non-INSERT/UPDATE queries (SELECT, DELETE, TRUNCATE) are rejected in static + // mode + // Test that SELECT queries are rejected + List rows = Collections.singletonList(Row.of("test1")); + + String selectQuery = String.format("SELECT * FROM %s.test_table WHERE id = ?", KEYSPACE); + CqlSinkConfig selectConfig = CqlSinkConfig.forRow().withQuery(selectQuery); + + assertThatThrownBy(() -> executeSinkAndWait(rows, selectConfig, ROW_S)) + .isInstanceOf(org.apache.flink.runtime.client.JobExecutionException.class) + .hasStackTraceContaining("Static mode only supports INSERT and UPDATE queries"); + + // Test that DELETE queries are rejected + String deleteQuery = String.format("DELETE FROM %s.test_table WHERE id = ?", KEYSPACE); + CqlSinkConfig deleteConfig = CqlSinkConfig.forRow().withQuery(deleteQuery); + + assertThatThrownBy(() -> executeSinkAndWait(rows, deleteConfig, ROW_S)) + .isInstanceOf(org.apache.flink.runtime.client.JobExecutionException.class) + .hasStackTraceContaining("Static mode only supports INSERT and UPDATE queries"); + + // Test that TRUNCATE queries are rejected + String truncateQuery = String.format("TRUNCATE %s.test_table", KEYSPACE); + CqlSinkConfig truncateConfig = CqlSinkConfig.forRow().withQuery(truncateQuery); + + // Use a dummy row since empty collections are not allowed + assertThatThrownBy(() -> executeSinkAndWait(rows, truncateConfig, ROW_S)) + .isInstanceOf(org.apache.flink.runtime.client.JobExecutionException.class) + .hasStackTraceContaining("Static mode only supports INSERT and UPDATE queries"); + } + + @Test + void testDynamicModeWithTableResolver() throws Exception { + // Tests dynamic mode routing records to different tables based on id parity + // Create two tables for dynamic routing + String createEvenTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.dynamic_even (" + + "id int PRIMARY KEY, " + + "name text, " + + "category text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createEvenTable); + + String createOddTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.dynamic_odd (" + + "id int PRIMARY KEY, " + + "name text, " + + "category text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createOddTable); + + // Create dynamic TableResolver that routes based on id + TableResolver tableResolver = new DynamicTableResolver(KEYSPACE); + + // Create ColumnValueResolver for INSERT + ColumnValueResolver columnResolver = new DynamicColumnResolver(); + + // Create SinkPluggable + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(tableResolver) + .withColumnValueResolver(columnResolver) + .build(); + + // Test data with mixed even/odd ids + List rows = new ArrayList<>(); + rows.add(Row.of(1, "odd1", "cat1")); + rows.add(Row.of(2, "even1", "cat2")); + rows.add(Row.of(3, "odd2", "cat3")); + rows.add(Row.of(4, "even2", "cat4")); + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + TypeInformation rowTypeInfo = Types.ROW(Types.INT, Types.STRING, Types.STRING); + executeSinkAndWait(rows, config, rowTypeInfo); + + // Verify odd table + String selectOdd = String.format("SELECT * FROM %s.dynamic_odd", KEYSPACE); + ResultSet rsOdd = cassandraTestEnvironment.executeRequestWithTimeout(selectOdd); + List oddRows = rsOdd.all(); + assertThat(oddRows).hasSize(2); + assertThat( + oddRows.stream() + .map(r -> r.getInt("id")) + .collect(java.util.stream.Collectors.toList())) + .containsExactlyInAnyOrder(1, 3); + + // Verify even table + String selectEven = String.format("SELECT * FROM %s.dynamic_even", KEYSPACE); + ResultSet rsEven = cassandraTestEnvironment.executeRequestWithTimeout(selectEven); + List evenRows = rsEven.all(); + assertThat(evenRows).hasSize(2); + assertThat( + evenRows.stream() + .map(r -> r.getInt("id")) + .collect(java.util.stream.Collectors.toList())) + .containsExactlyInAnyOrder(2, 4); + } + + @Test + void testDynamicModeWithColumnValueResolver() throws Exception { + // Tests dynamic mode with selective column inclusion based on record content + // Create table for selective column insertion test + String createTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.selective_columns (" + + "id int PRIMARY KEY, " + + "required_field text, " + + "optional_field text, " + + "computed_field text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createTable); + + // Create ColumnValueResolver that selectively includes columns + ColumnValueResolver selectiveResolver = new SelectiveColumnResolver(); + + // Create TableResolver for fixed table + TableResolver fixedTableResolver = + new FixedTableResolver(KEYSPACE, "selective_columns"); + + // Create SinkPluggable with TableResolver and ColumnValueResolver + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(fixedTableResolver) + .withColumnValueResolver(selectiveResolver) + .build(); + + // Test data: (id, required_field, should_include_optional) + List rows = new ArrayList<>(); + rows.add(Row.of(1, "req1", true)); // Include optional field + rows.add(Row.of(2, "req2", false)); // Skip optional field + rows.add(Row.of(3, "req3", true)); // Include optional field + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + TypeInformation rowTypeInfo = Types.ROW(Types.INT, Types.STRING, Types.BOOLEAN); + executeSinkAndWait(rows, config, rowTypeInfo); + + // Verify records were inserted with selective columns + String selectQuery = String.format("SELECT * FROM %s.selective_columns", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + List results = rs.all(); + assertThat(results).hasSize(3); + + // Check each record + for (com.datastax.driver.core.Row row : results) { + int id = row.getInt("id"); + assertThat(row.getString("required_field")).isEqualTo("req" + id); + assertThat(row.getString("computed_field")).isEqualTo("computed_" + id); + + if (id == 1 || id == 3) { + // Should have optional field + assertThat(row.getString("optional_field")).isEqualTo("optional_" + id); + } else { + // Should not have optional field (will be null) + assertThat(row.getString("optional_field")).isNull(); + } + } + } + + @Test + void testStatementCustomizer() throws Exception { + // Tests StatementCustomizer for setting consistency levels and idempotency based on record + // content + // Create table for statement customization test + String createTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.customized_statements (" + + "id int PRIMARY KEY, " + + "data text, " + + "priority text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createTable); + + // Test with dynamic mode and statement customizer + List rows = new ArrayList<>(); + rows.add(Row.of(1, "data1", "high")); + rows.add(Row.of(2, "data2", "low")); + rows.add(Row.of(3, "data3", "high")); + + // Create TableResolver for fixed table + TableResolver fixedTableResolver = + new FixedTableResolver(KEYSPACE, "customized_statements"); + + // Create ColumnValueResolver for INSERT + ColumnValueResolver insertResolver = new InsertAllColumnsResolver(); + + // Create SinkPluggable with StatementCustomizer + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(fixedTableResolver) + .withColumnValueResolver(insertResolver) + .withStatementCustomizer(new ConsistencyLevelCustomizer()) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + TypeInformation rowTypeInfo = Types.ROW(Types.INT, Types.STRING, Types.STRING); + executeSinkAndWait(rows, config, rowTypeInfo); + + // Verify records were inserted + String selectQuery = String.format("SELECT * FROM %s.customized_statements", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + List results = rs.all(); + assertThat(results).hasSize(3); + + // Verify data is correct + for (com.datastax.driver.core.Row row : results) { + int id = row.getInt("id"); + assertThat(row.getString("data")).isEqualTo("data" + id); + String expectedPriority = (id == 2) ? "low" : "high"; + assertThat(row.getString("priority")).isEqualTo(expectedPriority); + } + } + + @Test + void testRequestConfiguration() throws Exception { + // Tests RequestConfiguration with limited concurrent requests and retry settings + // Create table for request configuration test + String createTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.request_config_test (" + + "id int PRIMARY KEY, " + + "data text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createTable); + + // Create a larger dataset to test concurrent request limiting + List rows = new ArrayList<>(); + for (int i = 1; i <= 100; i++) { + rows.add(Row.of(i, "data" + i)); + } + + String insertQuery = + String.format( + "INSERT INTO %s.request_config_test (id, data) VALUES (?, ?)", KEYSPACE); + + // Configure with limited concurrent requests and retries + RequestConfiguration requestConfig = + RequestConfiguration.builder() + .setMaxConcurrentRequests(5) // Limit concurrent requests to 5 + .setMaxRetries(2) // Retry failed requests up to 2 times + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + + executeSinkAndWaitWithRequestConfig(rows, config, requestConfig, ROW_I_S); + + // Verify all records were inserted + String selectQuery = String.format("SELECT * FROM %s.request_config_test", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + List results = rs.all(); + assertThat(results).hasSize(100); + + // Verify data integrity + for (com.datastax.driver.core.Row row : results) { + int id = row.getInt("id"); + assertThat(row.getString("data")).isEqualTo("data" + id); + } + } + + // Static CqlClauseResolver implementations for testing + public static class DynamicTTLResolver implements CqlClauseResolver { + @Override + public ClauseBindings applyTo(Insert insert, Row record) { + insert.using(ttl(bindMarker())); + return new ClauseBindings(new Object[] {record.getField(2)}, null); + } + } + + public static class DynamicUpdateResolver implements CqlClauseResolver { + @Override + public ClauseBindings applyTo(Update update, Row record) { + update.using(ttl(bindMarker())).onlyIf(eq("version", bindMarker())); + return new ClauseBindings( + new Object[] {record.getField(2)}, // TTL only (no timestamp for LWT) + new Object[] {record.getField(3)} // IF version + ); + } + } + + // Simple TableResolver for single table tests + public static class SingleTableResolver implements TableResolver { + private final String tableName; + + public SingleTableResolver(String tableName) { + this.tableName = tableName; + } + + @Override + public TableRef resolve(Row record) { + return new TableRef(KEYSPACE, tableName); + } + } + + // Simple ColumnValueResolver for dynamic TTL tests + public static class DynamicTTLColumnResolver implements ColumnValueResolver { + @Override + public Kind kind() { + return Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + String id = (String) record.getField(0); + String name = (String) record.getField(1); + return ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {id, name}); + } + } + + public static class DynamicUpdateColumnResolver implements ColumnValueResolver { + @Override + public Kind kind() { + return Kind.UPDATE; + } + + @Override + public ResolvedWrite resolve(Row record) { + String id = (String) record.getField(0); + String name = (String) record.getField(1); + return ResolvedWrite.update( + Arrays.asList("name"), new Object[] {name}, + Arrays.asList("id"), new Object[] {id}); + } + } + + @Test + void testCqlClauseResolverDynamicInsertTTL() throws Exception { + // Tests CqlClauseResolver for dynamic per-record TTL values in INSERT statements + // Create table for dynamic TTL test + String createTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.dynamic_ttl (" + + "id text PRIMARY KEY, " + + "name text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createTable); + + // Test records with different TTL values per record + List rows = + Arrays.asList( + Row.of("ttl1", "first", 5), // 5 second TTL + Row.of("ttl2", "second", 5) // 5 second TTL + ); + + // Create dynamic SinkPluggable with CqlClauseResolver + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(new SingleTableResolver("dynamic_ttl")) + .withColumnValueResolver(new DynamicTTLColumnResolver()) + .withCqlClauseResolver(new DynamicTTLResolver()) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + TypeInformation rowTypeInfo = Types.ROW(Types.STRING, Types.STRING, Types.INT); + executeSinkAndWait(rows, config, rowTypeInfo); + + // Verify both records were inserted with correct TTL + String selectQuery = + String.format( + "SELECT id, name, TTL(name) as ttl_value FROM %s.dynamic_ttl", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + List results = rs.all(); + assertThat(results).hasSize(2); + + // Verify TTL values are approximately correct (allowing for execution time) + for (com.datastax.driver.core.Row row : results) { + String id = row.getString("id"); + int ttlValue = row.getInt("ttl_value"); + if ("ttl1".equals(id) || "ttl2".equals(id)) { + assertThat(ttlValue).isLessThanOrEqualTo(5).isGreaterThan(0); + } + } + } + + @Test + void testCqlClauseResolverDynamicUpdateWithLWT() throws Exception { + // Tests CqlClauseResolver for dynamic UPDATE with TTL and IF conditions (LWT) + // Create table with version column for LWT + String createTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.versioned_data (" + + "id text PRIMARY KEY, " + + "name text, " + + "version int" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createTable); + + // Insert initial data + String initialInsert = + String.format( + "INSERT INTO %s.versioned_data (id, name, version) VALUES ('ver1', 'initial', 1)", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(initialInsert); + + // Test record with TTL and version condition (no timestamp for LWT) + List rows = + Collections.singletonList( + Row.of("ver1", "updated", 5, 1) // id, name, ttl, expected_version + ); + + // Create dynamic SinkPluggable with CqlClauseResolver for UPDATE + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(new SingleTableResolver("versioned_data")) + .withColumnValueResolver(new DynamicUpdateColumnResolver()) + .withCqlClauseResolver(new DynamicUpdateResolver()) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + TypeInformation rowTypeInfo = + Types.ROW(Types.STRING, Types.STRING, Types.INT, Types.INT); + executeSinkAndWait(rows, config, rowTypeInfo); + + // Verify the update succeeded with applied=true + String selectQuery = + String.format( + "SELECT id, name, version, TTL(name) as ttl_value FROM %s.versioned_data WHERE id = 'ver1'", + KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row).isNotNull(); + assertThat(row.getString("name")).isEqualTo("updated"); + assertThat(row.getInt("version")).isEqualTo(1); + + // TTL should be approximately 5 seconds + int ttl = row.getInt("ttl_value"); + assertThat(ttl).isLessThanOrEqualTo(5).isGreaterThan(0); + } + + @Test + void testTTLVerificationWithPreciseExpiration() throws Exception { + // Tests precise TTL functionality with verification of record expiration timing + // Create table for TTL precision test + String createTable = + String.format( + "CREATE TABLE IF NOT EXISTS %s.ttl_precision (" + + "id text PRIMARY KEY, " + + "data text" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createTable); + + // Insert record with precise TTL + List rows = Collections.singletonList(Row.of("ttl-test", "expires-fast")); + + String insertQuery = + String.format( + "INSERT INTO %s.ttl_precision (id, data) VALUES (?, ?) USING TTL 5", + KEYSPACE); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(insertQuery); + executeSinkAndWait(rows, config, ROW_S_S); + + // Immediately verify TTL is set correctly (should be close to 5 seconds) + String selectQuery = + String.format( + "SELECT id, data, TTL(data) as ttl_remaining FROM %s.ttl_precision WHERE id = 'ttl-test'", + KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + com.datastax.driver.core.Row row = rs.one(); + assertThat(row).isNotNull(); + assertThat(row.getString("data")).isEqualTo("expires-fast"); + + // TTL should be 5 or slightly less (due to execution time) + int ttl = row.getInt("ttl_remaining"); + assertThat(ttl).isLessThanOrEqualTo(5).isGreaterThan(0); + + // Wait for expiration (6 seconds to be safe) + Thread.sleep(6000); + + // Verify record has expired + rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + row = rs.one(); + assertThat(row).isNull(); // Should be null after TTL expiration + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraPojoSinkITCase.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraPojoSinkITCase.java new file mode 100644 index 0000000..b9befe7 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraPojoSinkITCase.java @@ -0,0 +1,767 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.cassandra.CassandraTestEnvironment; +import org.apache.flink.connector.cassandra.sink.config.PojoSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.streaming.connectors.cassandra.MapperOptions; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; +import com.datastax.driver.mapping.Mapper; +import com.datastax.driver.mapping.annotations.Column; +import com.datastax.driver.mapping.annotations.Table; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.connector.cassandra.CassandraTestEnvironment.KEYSPACE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Integration tests for NEW CassandraSink V2 with POJO types using PojoSinkConfig. */ +@Testcontainers +class CassandraPojoSinkITCase { + + public static class TestClusterBuilder extends ClusterBuilder { + private final String host; + private final int port; + + public TestClusterBuilder(String host, int port) { + this.host = host; + this.port = port; + } + + @Override + protected Cluster buildCluster(Cluster.Builder builder) { + return builder.addContactPoint(host).withPort(port).build(); + } + } + + @TestEnv MiniClusterTestEnvironment flinkTestEnvironment = new MiniClusterTestEnvironment(); + + private static final CassandraTestEnvironment cassandraTestEnvironment = + new CassandraTestEnvironment(false); + + private static Integer port; + private static String host; + + @BeforeAll + static void setupCassandra() throws Exception { + cassandraTestEnvironment.startUp(); + createTestTable(); + port = cassandraTestEnvironment.getPort(); + host = cassandraTestEnvironment.getHost(); + } + + @AfterAll + static void tearDownCassandra() throws Exception { + dropTestTable(); + cassandraTestEnvironment.tearDown(); + } + + @BeforeEach + void clearTable() { + String truncateQuery = String.format("TRUNCATE %s.test_users;", KEYSPACE); + try { + cassandraTestEnvironment.executeRequestWithTimeout(truncateQuery); + } catch (Exception ignored) { + + } + } + + private static void createTestTable() { + String createQuery = + String.format( + "CREATE TABLE IF NOT EXISTS %s.test_users (" + + "id text PRIMARY KEY, " + + "name text, " + + "age int" + + ");", + KEYSPACE); + cassandraTestEnvironment.executeRequestWithTimeout(createQuery); + } + + private static void dropTestTable() { + String dropQuery = String.format("DROP TABLE IF EXISTS %s.test_users;", KEYSPACE); + try { + cassandraTestEnvironment.executeRequestWithTimeout(dropQuery); + } catch (Exception ignored) { + + } + } + + @Test + void testNewPojoSinkWithPojoSinkConfig() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // Create test data + List users = new ArrayList<>(); + users.add(new TestUser("user1", "Alice", 25)); + users.add(new TestUser("user2", "Bob", 30)); + users.add(new TestUser("user3", "Charlie", 35)); + + // Create data stream + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + // Create PojoSinkConfig for the NEW sink + PojoSinkConfig config = new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + // Build the NEW CassandraSink V2 + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + // Add the sink + dataStream.sinkTo(sink); + + env.execute(); + + // Verify data was inserted + List rows = readAllRows(); + assertThat(rows).hasSize(3); + + // Verify each user + for (TestUser user : users) { + Row row = readUserById(user.getId()); + assertThat(row).isNotNull(); + assertThat(row.getString("name")).isEqualTo(user.getName()); + assertThat(row.getInt("age")).isEqualTo(user.getAge()); + } + } + + @Test + void testNewPojoSinkWithNullValues() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // Create test data with some nulls + List users = new ArrayList<>(); + TestUser userWithNullName = new TestUser("user4", null, 40); + users.add(userWithNullName); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + // Create PojoSinkConfig for the NEW sink + PojoSinkConfig config = new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + // Build the NEW CassandraSink V2 + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Verify data + Row row = readUserById("user4"); + assertThat(row).isNotNull(); + assertThat(row.isNull("name")).isTrue(); + assertThat(row.getInt("age")).isEqualTo(40); + } + + @Test + void testNewPojoSinkWithMapperOptions() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List users = new ArrayList<>(); + users.add(new TestUser("user5", "Eve", 28)); + users.add(new TestUser("user6", "Frank", 32)); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + // Create MapperOptions with TTL and consistency level + MapperOptions mapperOptions = + () -> + new Mapper.Option[] { + Mapper.Option.ttl(5), // 5 seconds TTL for testing + Mapper.Option.consistencyLevel(ConsistencyLevel.QUORUM) + }; + + // Create PojoSinkConfig with mapper options + PojoSinkConfig config = + new PojoSinkConfig<>(TestUser.class, KEYSPACE, mapperOptions); + + // Build the NEW CassandraSink V2 + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Verify exact count of records + List rows = readAllRows(); + assertThat(rows).hasSize(2); + + // Verify TTL was applied + String ttlQuery = + String.format( + "SELECT TTL(name) AS name_ttl, TTL(age) AS age_ttl FROM %s.test_users WHERE id='user5';", + KEYSPACE); + ResultSet ttlRs = cassandraTestEnvironment.executeRequestWithTimeout(ttlQuery); + Row ttlRow = ttlRs.one(); + assertThat(ttlRow).isNotNull(); + assertThat(ttlRow.getInt("name_ttl")) + .isBetween(1, 6); // Wider window to prevent CI flakiness + assertThat(ttlRow.getInt("age_ttl")).isBetween(1, 6); + + Row row = readUserById("user5"); + assertThat(row).isNotNull(); + assertThat(row.getString("name")).isEqualTo("Eve"); + } + + @Test + void testNewPojoSinkWithLargerBatch() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + // Create larger batch of data + List users = new ArrayList<>(); + for (int i = 0; i < 50; i++) { + users.add(new TestUser("id" + i, "User" + i, 20 + (i % 30))); + } + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + // Create PojoSinkConfig for the NEW sink + PojoSinkConfig config = new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + // Build the NEW CassandraSink V2 + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Verify all records were inserted + List rows = readAllRows(); + assertThat(rows).hasSize(50); + } + + @Test + void testTimestampMapperOption() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // Custom timestamp in microseconds + long customMicros = System.currentTimeMillis() * 1000L + 123; + + List users = new ArrayList<>(); + users.add(new TestUser("ts_user", "Alice", 25)); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + // Create MapperOptions with custom timestamp + MapperOptions timestampOptions = + () -> new Mapper.Option[] {Mapper.Option.timestamp(customMicros)}; + + PojoSinkConfig config = + new PojoSinkConfig<>(TestUser.class, KEYSPACE, timestampOptions); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Verify the custom timestamp was applied + String writetimeQuery = + String.format( + "SELECT writetime(name) AS wt FROM %s.test_users WHERE id='ts_user';", + KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(writetimeQuery); + Row row = rs.one(); + assertThat(row).isNotNull(); + assertThat(row.getLong("wt")).isEqualTo(customMicros); + } + + @Test + void testConcurrencyAndBackpressure() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(4); // High parallelism + + // Create a large batch of records + List users = new ArrayList<>(); + for (int i = 0; i < 500; i++) { + users.add(new TestUser("concurrent_" + i, "User" + i, 20 + (i % 50))); + } + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + PojoSinkConfig config = new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + // Configure with limited concurrency to exercise permit/flush path + RequestConfiguration requestConfig = + RequestConfiguration.builder() + .setMaxConcurrentRequests(2) // Very low to force queuing + .build(); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .setRequestConfiguration(requestConfig) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Verify all records were written despite concurrency limits + // Count manually since LIKE with COUNT can be problematic + int count = 0; + for (int i = 0; i < 500; i++) { + Row row = readUserById("concurrent_" + i); + if (row != null) { + count++; + } + } + assertThat(count).isEqualTo(500).as("All 500 records should be written"); + } + + /** Simple test POJO with Cassandra annotations. */ + @Table(keyspace = KEYSPACE, name = "test_users") + public static class TestUser implements Serializable { + @Column(name = "id") + private String id; + + @Column(name = "name") + private String name; + + @Column(name = "age") + private Integer age; + + public TestUser() { + // Default constructor required for DataStax mapper + } + + public TestUser(String id, String name, Integer age) { + this.id = id; + this.name = name; + this.age = age; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + } + + @Test + void testTTLExpiry() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List users = new ArrayList<>(); + users.add(new TestUser("ttl_user", "Temporary", 100)); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + // Create MapperOptions with short TTL (long enough for initial read) + MapperOptions mapperOptions = () -> new Mapper.Option[] {Mapper.Option.ttl(5)}; + + PojoSinkConfig config = + new PojoSinkConfig<>(TestUser.class, KEYSPACE, mapperOptions); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Wait for record to be written with retry mechanism + Row row = null; + long startWait = System.currentTimeMillis(); + while (row == null && (System.currentTimeMillis() - startWait) < 2000) { // 2 second timeout + row = readUserById("ttl_user"); + if (row == null) { + Thread.sleep(50); // Wait 50ms before retry + } + } + assertThat(row).isNotNull(); + assertThat(row.getString("name")).isEqualTo("Temporary"); + long startTime = System.currentTimeMillis(); + boolean expired = false; + while (System.currentTimeMillis() - startTime + < 8000) { // 8 second timeout (longer than 5s TTL) + Thread.sleep(500); + Row expiredRow = readUserById("ttl_user"); + if (expiredRow == null) { + expired = true; + break; + } + } + assertThat(expired).isTrue().as("Record should have expired after TTL"); + } + + @Test + void testSaveNullFieldsFalse() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // First, write a complete record + List users = new ArrayList<>(); + users.add(new TestUser("null_test", "Alice", 25)); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + PojoSinkConfig config = new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Verify initial state + Row row = readUserById("null_test"); + assertThat(row.getString("name")).isEqualTo("Alice"); + + // Now update with null name but saveNullFields(false) + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List updateUsers = new ArrayList<>(); + updateUsers.add(new TestUser("null_test", null, 26)); + + DataStream updateStream = + env.fromCollection(updateUsers, TypeInformation.of(TestUser.class)); + + MapperOptions noNullsOptions = + () -> new Mapper.Option[] {Mapper.Option.saveNullFields(false)}; + + PojoSinkConfig noNullsConfig = + new PojoSinkConfig<>(TestUser.class, KEYSPACE, noNullsOptions); + + CassandraSink noNullsSink = + CassandraSinkBuilder.newBuilder(noNullsConfig) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + updateStream.sinkTo(noNullsSink); + env.execute(); + + // Verify name was NOT overwritten with null + Row updatedRow = readUserById("null_test"); + assertThat(updatedRow.getString("name")).isEqualTo("Alice"); // Still Alice + assertThat(updatedRow.getInt("age")).isEqualTo(26); // Age updated + } + + @Test + void testSaveNullFieldsDefault() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // First, write a complete record + List users = new ArrayList<>(); + users.add(new TestUser("null_default", "Bob", 30)); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + PojoSinkConfig config = new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Verify initial state + Row row = readUserById("null_default"); + assertThat(row.getString("name")).isEqualTo("Bob"); + + // Now update with null name using default behavior + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List updateUsers = new ArrayList<>(); + updateUsers.add(new TestUser("null_default", null, 31)); + + DataStream updateStream = + env.fromCollection(updateUsers, TypeInformation.of(TestUser.class)); + + PojoSinkConfig updateConfig = + new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + CassandraSink updateSink = + CassandraSinkBuilder.newBuilder(updateConfig) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + updateStream.sinkTo(updateSink); + env.execute(); + + // Verify name WAS overwritten with null (tombstone) + Row updatedRow = readUserById("null_default"); + assertThat(updatedRow.isNull("name")).isTrue(); // Name is null + assertThat(updatedRow.getInt("age")).isEqualTo(31); // Age updated + } + + @Test + void testUpsertSemantics() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // Insert initial record + List users = new ArrayList<>(); + users.add(new TestUser("upsert_test", "Bob", 30)); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + PojoSinkConfig config = new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + env.execute(); + + // Verify initial state + Row row = readUserById("upsert_test"); + assertThat(row.getString("name")).isEqualTo("Bob"); + assertThat(row.getInt("age")).isEqualTo(30); + + // Update same record + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List updateUsers = new ArrayList<>(); + updateUsers.add(new TestUser("upsert_test", "Bobby", 31)); + + DataStream updateStream = + env.fromCollection(updateUsers, TypeInformation.of(TestUser.class)); + + CassandraSink updateSink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + updateStream.sinkTo(updateSink); + env.execute(); + + // Verify update and check writetime monotonicity + Row updatedRow = readUserById("upsert_test"); + assertThat(updatedRow.getString("name")).isEqualTo("Bobby"); + assertThat(updatedRow.getInt("age")).isEqualTo(31); + + // Verify writetime increased (proving actual update happened) + String writetimeQuery = + String.format( + "SELECT writetime(name) AS wt FROM %s.test_users WHERE id='upsert_test';", + KEYSPACE); + ResultSet firstRs = cassandraTestEnvironment.executeRequestWithTimeout(writetimeQuery); + long firstWritetime = firstRs.one().getLong("wt"); + + // Do another update to verify monotonicity + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + List finalUpdate = new ArrayList<>(); + finalUpdate.add(new TestUser("upsert_test", "Robert", 32)); + DataStream finalStream = + env.fromCollection(finalUpdate, TypeInformation.of(TestUser.class)); + CassandraSink finalSink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + finalStream.sinkTo(finalSink); + env.execute(); + + ResultSet secondRs = cassandraTestEnvironment.executeRequestWithTimeout(writetimeQuery); + long secondWritetime = secondRs.one().getLong("wt"); + assertThat(secondWritetime) + .isGreaterThan(firstWritetime) + .as("Writetime should increase on update"); + } + + @Test + void testNullPartitionKeyFailure() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // Create a record with null partition key + List users = new ArrayList<>(); + users.add(new TestUser(null, "Invalid", 99)); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(TestUser.class)); + + PojoSinkConfig config = new PojoSinkConfig<>(TestUser.class, KEYSPACE, null); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + + // Execute should throw exception - the null partition key causes InvalidQueryException + // which is classified as fatal with a specific message + assertThatThrownBy(() -> env.execute()) + .hasStackTraceContaining("Encountered a non-recoverable InvalidQueryException") + .hasStackTraceContaining("Invalid null value"); + } + + @Test + void testMissingTableFailure() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List users = new ArrayList<>(); + users.add(new NonExistentTableUser("test_missing", "Test", 25)); + + DataStream dataStream = + env.fromCollection(users, TypeInformation.of(NonExistentTableUser.class)); + + PojoSinkConfig config = + new PojoSinkConfig<>(NonExistentTableUser.class, KEYSPACE, null); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder(host, port)) + .build(); + + dataStream.sinkTo(sink); + + // Execute should throw exception for missing table + assertThatThrownBy(() -> env.execute()) + .hasStackTraceContaining("does not exist in keyspace"); + } + + /** Test POJO with a non-existent table name to test failure scenarios. */ + @Table(keyspace = KEYSPACE, name = "non_existent_table") + public static class NonExistentTableUser implements Serializable { + @Column(name = "id") + private String id; + + @Column(name = "name") + private String name; + + @Column(name = "age") + private Integer age; + + public NonExistentTableUser() { + // Default constructor required for DataStax mapper + } + + public NonExistentTableUser(String id, String name, Integer age) { + this.id = id; + this.name = name; + this.age = age; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + } + + // ============== Helper Methods ============== + + private List readAllRows() { + String selectQuery = String.format("SELECT * FROM %s.test_users;", KEYSPACE); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + return rs.all(); + } + + private Row readUserById(String id) { + String selectQuery = + String.format("SELECT * FROM %s.test_users WHERE id = '%s';", KEYSPACE, id); + ResultSet rs = cassandraTestEnvironment.executeRequestWithTimeout(selectQuery); + return rs.one(); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkBuilderTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkBuilderTest.java new file mode 100644 index 0000000..69df0fc --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkBuilderTest.java @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.PojoSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration; +import org.apache.flink.connector.cassandra.sink.exception.CassandraFailureHandler; +import org.apache.flink.connector.cassandra.sink.planner.SinkPluggable; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.streaming.connectors.cassandra.SimpleMapperOptions; +import org.apache.flink.types.Row; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.QueryBuilder; +import com.datastax.driver.core.querybuilder.Update; +import com.datastax.driver.mapping.annotations.Column; +import com.datastax.driver.mapping.annotations.Table; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link CassandraSinkBuilder}. */ +public class CassandraSinkBuilderTest { + + /** Test POJO class with Cassandra annotations. */ + @Table(keyspace = "test_keyspace", name = "test_table") + public static class TestPojo { + @Column(name = "id") + private String id; + + @Column(name = "name") + private String name; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + } + + private static class TestClusterBuilder extends ClusterBuilder { + @Override + protected Cluster buildCluster(Cluster.Builder builder) { + return builder.addContactPoint("localhost").build(); + } + } + + @Test + void testConnectionConfiguration() { + CqlSinkConfig config = + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + + // Test 1: Host only with default port - verify ALL defaults are set + CassandraSink sink1 = + CassandraSinkBuilder.newBuilder(config).setHost("localhost").build(); + assertThat(sink1).isNotNull(); + assertThat(sink1.getClusterBuilder()).isNotNull(); + assertThat(sink1.getSinkConfig()).isSameAs(config); + assertThat(sink1.getFailureHandler()).isNotNull(); + assertThat(sink1.getFailureHandler()).isExactlyInstanceOf(CassandraFailureHandler.class); + assertThat(sink1.getRequestConfiguration()).isNotNull(); + assertThat(sink1.getRequestConfiguration().getMaxRetries()).isEqualTo(0); + assertThat(sink1.getRequestConfiguration().getMaxConcurrentRequests()) + .isEqualTo(Integer.MAX_VALUE); + assertThat(sink1.getRequestConfiguration().getMaxTimeout()) + .isEqualTo(Duration.ofMinutes(1)); + assertThat(sink1.getRequestConfiguration().getFlushTimeout()) + .isEqualTo(Duration.ofSeconds(30)); + + // Test 2: Host with custom port - verify all components are set + CassandraSink sink2 = + CassandraSinkBuilder.newBuilder(config).setHost("localhost").setPort(9999).build(); + assertThat(sink2).isNotNull(); + assertThat(sink2.getClusterBuilder()).isNotNull(); + assertThat(sink2.getSinkConfig()).isSameAs(config); + assertThat(sink2.getFailureHandler()).isNotNull(); + assertThat(sink2.getFailureHandler()).isExactlyInstanceOf(CassandraFailureHandler.class); + assertThat(sink2.getRequestConfiguration()).isNotNull(); + assertThat(sink2.getRequestConfiguration().getMaxRetries()).isEqualTo(0); + assertThat(sink2.getRequestConfiguration().getMaxConcurrentRequests()) + .isEqualTo(Integer.MAX_VALUE); + assertThat(sink2.getRequestConfiguration().getMaxTimeout()) + .isEqualTo(Duration.ofMinutes(1)); + assertThat(sink2.getRequestConfiguration().getFlushTimeout()) + .isEqualTo(Duration.ofSeconds(30)); + + // Test 3: Custom ClusterBuilder - verify custom builder is used + TestClusterBuilder customClusterBuilder = new TestClusterBuilder(); + CassandraSink sink3 = + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(customClusterBuilder) + .build(); + assertThat(sink3).isNotNull(); + assertThat(sink3.getClusterBuilder()).isSameAs(customClusterBuilder); + assertThat(sink3.getSinkConfig()).isSameAs(config); + assertThat(sink3.getFailureHandler()).isNotNull(); + assertThat(sink3.getFailureHandler()).isExactlyInstanceOf(CassandraFailureHandler.class); + assertThat(sink3.getRequestConfiguration()).isNotNull(); + assertThat(sink3.getRequestConfiguration().getMaxRetries()).isEqualTo(0); + assertThat(sink3.getRequestConfiguration().getMaxConcurrentRequests()) + .isEqualTo(Integer.MAX_VALUE); + assertThat(sink3.getRequestConfiguration().getMaxTimeout()) + .isEqualTo(Duration.ofMinutes(1)); + assertThat(sink3.getRequestConfiguration().getFlushTimeout()) + .isEqualTo(Duration.ofSeconds(30)); + } + + @Test + void testBuilderValidation() { + CqlSinkConfig config = + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + + // Connection validation + + // Test 1: No connection info should fail + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .build()) // No connection info set + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cassandra connection information must be supplied"); + + // Test 2: ClusterBuilder then host should fail + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder()) + .setHost("localhost") // This should fail + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("ClusterBuilder was already set"); + + // Test 3: ClusterBuilder then port should fail + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(new TestClusterBuilder()) + .setPort(9999) // This should fail + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("ClusterBuilder was already set"); + + // Test 4: Host then ClusterBuilder should fail + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setHost("localhost") + .setClusterBuilder( + new TestClusterBuilder()) // This should fail + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Connection information was already set"); + + // Test 5: Port then ClusterBuilder should fail + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setPort(9999) + .setClusterBuilder( + new TestClusterBuilder()) // This should fail + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Connection information was already set"); + + // Test 6: Port without host should fail on build + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setPort(9999) // Port without host + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cassandra connection information must be supplied"); + + // Port validation + + // Test 7: Invalid port - zero + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setHost("localhost") + .setPort(0) // Invalid port + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("port must be between 1 and 65535"); + + // Test 8: Invalid port - negative + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setHost("localhost") + .setPort(-1) // Invalid port + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("port must be between 1 and 65535"); + + // Test 9: Invalid port - too high + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setHost("localhost") + .setPort(65536) // Invalid port + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("port must be between 1 and 65535"); + + // Null/empty validation + + // Test 10: Null host + assertThatThrownBy(() -> CassandraSinkBuilder.newBuilder(config).setHost(null).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("host must not be null or empty"); + + // Test 11: Empty host + assertThatThrownBy(() -> CassandraSinkBuilder.newBuilder(config).setHost("").build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("host must not be null or empty"); + + // Test 12: Null ClusterBuilder + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setClusterBuilder(null) + .build()) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("clusterBuilder must not be null"); + + // Test 13: Null FailureHandler + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setHost("localhost") + .setFailureHandler(null) + .build()) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("failureHandler must not be null"); + + // Test 14: Null RequestConfiguration + assertThatThrownBy( + () -> + CassandraSinkBuilder.newBuilder(config) + .setHost("localhost") + .setRequestConfiguration(null) + .build()) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("requestConfig must not be null"); + } + + @Test + void testDefaultsInjection() { + CqlSinkConfig config = + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config).setHost("localhost").build(); + + // Verify defaults were injected + assertThat(sink.getFailureHandler()).isNotNull(); + assertThat(sink.getFailureHandler()).isExactlyInstanceOf(CassandraFailureHandler.class); + assertThat(sink.getRequestConfiguration()).isNotNull(); + // Check all default RequestConfiguration values + assertThat(sink.getRequestConfiguration().getMaxRetries()).isEqualTo(0); + assertThat(sink.getRequestConfiguration().getMaxConcurrentRequests()) + .isEqualTo(Integer.MAX_VALUE); + assertThat(sink.getRequestConfiguration().getMaxTimeout()).isEqualTo(Duration.ofMinutes(1)); + assertThat(sink.getRequestConfiguration().getFlushTimeout()) + .isEqualTo(Duration.ofSeconds(30)); + } + + @Test + void testCustomConfigurationPropagation() { + CqlSinkConfig config = + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + CassandraFailureHandler customHandler = new CassandraFailureHandler(); + RequestConfiguration customRequestConfig = + RequestConfiguration.builder() + .setMaxRetries(5) + .setMaxConcurrentRequests(100) + .setMaxTimeout(Duration.ofSeconds(10)) + .setFlushTimeout(Duration.ofSeconds(5)) + .build(); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(config) + .setHost("localhost") + .setFailureHandler(customHandler) + .setRequestConfiguration(customRequestConfig) + .build(); + + // Verify custom configurations were propagated + assertThat(sink.getFailureHandler()).isSameAs(customHandler); + assertThat(sink.getRequestConfiguration()).isSameAs(customRequestConfig); + assertThat(sink.getSinkConfig()).isSameAs(config); + + // Verify the actual custom values in RequestConfiguration + assertThat(sink.getRequestConfiguration().getMaxRetries()).isEqualTo(5); + assertThat(sink.getRequestConfiguration().getMaxConcurrentRequests()).isEqualTo(100); + assertThat(sink.getRequestConfiguration().getMaxTimeout()) + .isEqualTo(Duration.ofSeconds(10)); + assertThat(sink.getRequestConfiguration().getFlushTimeout()) + .isEqualTo(Duration.ofSeconds(5)); + } + + @Test + void testCqlBuilderWithUnsetModeFails() { + // Create config in UNSET mode (no query, no pluggable) + CqlSinkConfig config = CqlSinkConfig.forRow(); + + assertThatThrownBy( + () -> CassandraSinkBuilder.newBuilder(config).setHost("localhost").build()) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "CQL sink configuration requires exactly one mode: either call withQuery(...) for STATIC mode, or withPluggable(...) for DYNAMIC mode."); + } + + @Test + void testCqlBuilderModes() { + // Static mode with query + CqlSinkConfig staticConfig = + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + CassandraSink staticSink = + CassandraSinkBuilder.newBuilder(staticConfig).setHost("localhost").build(); + assertThat(staticSink).isNotNull(); + assertThat(staticSink.getSinkConfig()).isSameAs(staticConfig); + + // Dynamic mode with pluggable - needs resolvers + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(record -> new TableRef("keyspace", "table")) + .withColumnValueResolver( + new ColumnValueResolver() { + @Override + public Kind kind() { + return Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + return ResolvedWrite.insert( + Arrays.asList("id", "name"), + new Object[] { + record.getField(0), record.getField(1) + }); + } + }) + .build(); + + CqlSinkConfig dynamicConfig = CqlSinkConfig.forRow().withPluggable(pluggable); + CassandraSink dynamicSink = + CassandraSinkBuilder.newBuilder(dynamicConfig).setHost("localhost").build(); + assertThat(dynamicSink).isNotNull(); + assertThat(dynamicSink.getSinkConfig()).isSameAs(dynamicConfig); + } + + @Test + void testPojoBuilderConfiguration() { + PojoSinkConfig pojoConfig = + new PojoSinkConfig<>(TestPojo.class, "test_keyspace", new SimpleMapperOptions()); + + CassandraSink sink = + CassandraSinkBuilder.newBuilder(pojoConfig) + .setHost("localhost") + .setPort(9042) + .build(); + + assertThat(sink).isNotNull(); + assertThat(sink.getSinkConfig()).isSameAs(pojoConfig); + assertThat(sink.getClusterBuilder()).isNotNull(); + assertThat(sink.getFailureHandler()).isNotNull(); + assertThat(sink.getFailureHandler()).isExactlyInstanceOf(CassandraFailureHandler.class); + assertThat(sink.getRequestConfiguration()).isNotNull(); + // Check default RequestConfiguration values + assertThat(sink.getRequestConfiguration().getMaxRetries()).isEqualTo(0); + assertThat(sink.getRequestConfiguration().getMaxConcurrentRequests()) + .isEqualTo(Integer.MAX_VALUE); + assertThat(sink.getRequestConfiguration().getMaxTimeout()).isEqualTo(Duration.ofMinutes(1)); + assertThat(sink.getRequestConfiguration().getFlushTimeout()) + .isEqualTo(Duration.ofSeconds(30)); + } + + @Test + void testSinkPluggableValidation() { + // Test 1: SinkPluggable without resolvers should fail + assertThatThrownBy(() -> SinkPluggable.builder().build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("TableResolver cannot be null"); + + // Test 2: SinkPluggable with only TableResolver should fail + assertThatThrownBy( + () -> + SinkPluggable.builder() + .withTableResolver( + record -> new TableRef("keyspace", "table")) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("ColumnValueResolver cannot be null"); + + // Test 3: Valid minimal SinkPluggable (required resolvers only) + SinkPluggable minimalPluggable = + SinkPluggable.builder() + .withTableResolver(record -> new TableRef("keyspace", "table")) + .withColumnValueResolver( + new ColumnValueResolver() { + @Override + public Kind kind() { + return Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + return ResolvedWrite.insert( + Arrays.asList("id", "name"), + new Object[] { + record.getField(0), record.getField(1) + }); + } + }) + .build(); + assertThat(minimalPluggable).isNotNull(); + + // Test 4: SinkPluggable with all optional resolvers + SinkPluggable fullPluggable = + SinkPluggable.builder() + .withTableResolver(record -> new TableRef("keyspace", "table")) + .withColumnValueResolver( + new ColumnValueResolver() { + @Override + public Kind kind() { + return Kind.UPDATE; + } + + @Override + public ResolvedWrite resolve(Row record) { + return ResolvedWrite.update( + Arrays.asList("name"), + new Object[] {record.getField(1)}, + Arrays.asList("id"), + new Object[] {record.getField(0)}); + } + }) + .withCqlClauseResolver( + new CqlClauseResolver() { + @Override + public ClauseBindings applyTo(Insert insert, Row record) { + insert.using(QueryBuilder.ttl(3600)); + return ClauseBindings.empty(); + } + + @Override + public ClauseBindings applyTo(Update update, Row record) { + update.using(QueryBuilder.ttl(3600)); + return ClauseBindings.empty(); + } + }) + .withStatementCustomizer( + (StatementCustomizer) + (statement, record) -> { + statement.setConsistencyLevel(ConsistencyLevel.QUORUM); + statement.setIdempotent(true); + }) + .build(); + assertThat(fullPluggable).isNotNull(); + + // Test 5: Verify we can use the pluggable in a sink + CqlSinkConfig pluggableConfig = CqlSinkConfig.forRow().withPluggable(fullPluggable); + CassandraSink sink = + CassandraSinkBuilder.newBuilder(pluggableConfig).setHost("localhost").build(); + assertThat(sink).isNotNull(); + assertThat(sink.getSinkConfig()).isSameAs(pluggableConfig); + } + + @Test + void testValidPortRangeAccepted() { + CassandraSinkBuilder.CqlBuilder builder1 = + CassandraSinkBuilder.newBuilder( + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)")); + builder1.setPort(1); // Minimum valid + assertThat(builder1).isNotNull(); + + CassandraSinkBuilder.CqlBuilder builder2 = + CassandraSinkBuilder.newBuilder( + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)")); + builder2.setPort(65535); // Maximum valid + assertThat(builder2).isNotNull(); + + CassandraSinkBuilder.CqlBuilder builder3 = + CassandraSinkBuilder.newBuilder( + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)")); + builder3.setPort(9042); // Default Cassandra port + assertThat(builder3).isNotNull(); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriterITCase.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriterITCase.java new file mode 100644 index 0000000..e018db2 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriterITCase.java @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink; + +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.connector.cassandra.CassandraTestEnvironment; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration; +import org.apache.flink.connector.cassandra.sink.exception.CassandraFailureHandler; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.runtime.mailbox.SyncMailboxExecutor; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.FlinkRuntimeException; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.exceptions.InvalidQueryException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.time.Duration; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Minimal integration tests for {@link CassandraSinkWriter} with real Cassandra. Goal: prove the + * real writer + semaphore/mailbox loop works end-to-end without deadlock. Heavy behavior testing + * belongs in CassandraSinkWriterTest with mocks. + */ +@Testcontainers +class CassandraSinkWriterITCase { + + private static final CassandraTestEnvironment cassandraTestEnvironment = + new CassandraTestEnvironment(false); + + @RegisterExtension + static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension(); + + private static final String KEYSPACE = "it_ks"; + private static final String TABLE = "it_smoke"; + private static final String CREATE_KEYSPACE_QUERY = + "CREATE KEYSPACE IF NOT EXISTS " + + KEYSPACE + + " WITH replication = {'class':'SimpleStrategy','replication_factor':1}"; + private static final String CREATE_TABLE_QUERY = + "CREATE TABLE IF NOT EXISTS " + + KEYSPACE + + "." + + TABLE + + " (pk int PRIMARY KEY, v text)"; + private static final String INSERT_QUERY = + "INSERT INTO " + KEYSPACE + "." + TABLE + " (pk, v) VALUES (?, ?)"; + private static final String SELECT_COUNT_QUERY = + "SELECT COUNT(*) FROM " + KEYSPACE + "." + TABLE; + private static final String TRUNCATE_QUERY = "TRUNCATE " + KEYSPACE + "." + TABLE; + + private static Session session; + private ClusterBuilder clusterBuilder; + private SinkWriterMetricGroup metricGroup; + private SyncMailboxExecutor mailboxExecutor; + + @BeforeAll + static void setupCassandra() throws Exception { + cassandraTestEnvironment.startUp(); + session = cassandraTestEnvironment.getSession(); + session.execute(CREATE_KEYSPACE_QUERY); + session.execute(CREATE_TABLE_QUERY); + } + + @AfterAll + static void tearDownCassandra() throws Exception { + if (session != null) { + session.close(); + } + cassandraTestEnvironment.tearDown(); + } + + @BeforeEach + void setUp() throws Exception { + try { + session.execute(TRUNCATE_QUERY); + } catch (Exception ignored) { + } + clusterBuilder = + new ClusterBuilder() { + @Override + protected Cluster buildCluster(Cluster.Builder builder) { + return builder.addContactPoint(cassandraTestEnvironment.getHost()) + .withPort(cassandraTestEnvironment.getPort()) + .build(); + } + }; + + setupMetricGroup(); + mailboxExecutor = new SyncMailboxExecutor(); + } + + @AfterEach + void tearDown() throws Exception { + try { + session.execute(TRUNCATE_QUERY); + } catch (Exception ignored) { + + } + } + + @Test + void testBackpressureSmoke_maxConcurrent1() throws Exception { + RequestConfiguration requestConfig = + RequestConfiguration.builder() + .setMaxConcurrentRequests(1) + .setMaxRetries(1) + .setMaxTimeout(Duration.ofSeconds(10)) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(INSERT_QUERY); + + try (CassandraSinkWriter sinkWriter = + new CassandraSinkWriter<>( + requestConfig, + config, + clusterBuilder, + new CassandraFailureHandler(), + mailboxExecutor, + metricGroup)) { + + // When you fire 3-4 writes quickly, then flush() + sinkWriter.write(Row.of(1, "v1"), mock(SinkWriter.Context.class)); + sinkWriter.write(Row.of(2, "v2"), mock(SinkWriter.Context.class)); + sinkWriter.write(Row.of(3, "v3"), mock(SinkWriter.Context.class)); + sinkWriter.write(Row.of(4, "v4"), mock(SinkWriter.Context.class)); + + sinkWriter.flush(false); + assertThat(getRowCount()).isEqualTo(4); + } + } + + @Test + void testFlushWaitsForCompletion() throws Exception { + // Given writer with maxConcurrentRequests=2 + RequestConfiguration requestConfig = + RequestConfiguration.builder() + .setMaxConcurrentRequests(2) + .setMaxRetries(1) + .setMaxTimeout(Duration.ofSeconds(10)) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(INSERT_QUERY); + + try (CassandraSinkWriter sinkWriter = + new CassandraSinkWriter<>( + requestConfig, + config, + clusterBuilder, + new CassandraFailureHandler(), + mailboxExecutor, + metricGroup)) { + int k = 5; + for (int i = 0; i < k; i++) { + sinkWriter.write(Row.of(i, "value" + i), mock(SinkWriter.Context.class)); + } + + sinkWriter.flush(false); + + // Then after flush(), SELECT COUNT(*) = k + assertThat(getRowCount()).isEqualTo(k); + } + } + + @Test + void testCloseIsIdempotent() throws Exception { + // Given some writes in flight + RequestConfiguration requestConfig = + RequestConfiguration.builder() + .setMaxConcurrentRequests(2) + .setMaxRetries(1) + .setMaxTimeout(Duration.ofSeconds(10)) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(INSERT_QUERY); + + CassandraSinkWriter sinkWriter = + new CassandraSinkWriter<>( + requestConfig, + config, + clusterBuilder, + new CassandraFailureHandler(), + mailboxExecutor, + metricGroup); + + sinkWriter.write(Row.of(1, "test1"), mock(SinkWriter.Context.class)); + sinkWriter.write(Row.of(2, "test2"), mock(SinkWriter.Context.class)); + + // When close() then close() again + sinkWriter.close(); + sinkWriter.close(); // Should be idempotent + + // Then no exception; rows are persisted + assertThat(getRowCount()).isEqualTo(2); + + // And subsequent write() throws (closed state) - expect RuntimeException due to closed + // cluster + assertThatThrownBy( + () -> sinkWriter.write(Row.of(3, "fail"), mock(SinkWriter.Context.class))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Cannot execute mail"); + } + + @Test + void testFatalErrorSurfacesOnce() throws Exception { + // Given config targeting a non-existent table + String badInsertQuery = "INSERT INTO " + KEYSPACE + ".non_existent (pk, v) VALUES (?, ?)"; + + RequestConfiguration requestConfig = + RequestConfiguration.builder() + .setMaxConcurrentRequests(1) + .setMaxRetries(1) + .setMaxTimeout(Duration.ofSeconds(5)) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withQuery(badInsertQuery); + + try (CassandraSinkWriter sinkWriter = + new CassandraSinkWriter<>( + requestConfig, + config, + clusterBuilder, + new CassandraFailureHandler(), + mailboxExecutor, + metricGroup)) { + + assertThatThrownBy( + () -> + sinkWriter.write( + Row.of(1, "test"), mock(SinkWriter.Context.class))) + .isInstanceOf(FlinkRuntimeException.class) + .hasRootCauseInstanceOf(InvalidQueryException.class); + } + } + + private long getRowCount() { + ResultSet result = session.execute(SELECT_COUNT_QUERY); + return result.one().getLong(0); + } + + private void setupMetricGroup() { + metricGroup = mock(SinkWriterMetricGroup.class); + OperatorIOMetricGroup ioMetricGroup = mock(OperatorIOMetricGroup.class); + + when(metricGroup.getIOMetricGroup()).thenReturn(ioMetricGroup); + when(ioMetricGroup.getNumRecordsOutCounter()).thenReturn(new SimpleCounter()); + when(metricGroup.getNumRecordsOutErrorsCounter()).thenReturn(new SimpleCounter()); + when(metricGroup.counter(any())).thenReturn(new SimpleCounter()); + when(metricGroup.histogram(any(), any())).thenReturn(mock(Histogram.class)); + } + + /** Simple counter for metrics. */ + private static class SimpleCounter implements Counter { + private long count = 0; + + @Override + public void inc() { + count++; + } + + @Override + public void inc(long n) { + count += n; + } + + @Override + public void dec() { + count--; + } + + @Override + public void dec(long n) { + count -= n; + } + + @Override + public long getCount() { + return count; + } + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriterTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriterTest.java new file mode 100644 index 0000000..782130e --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/CassandraSinkWriterTest.java @@ -0,0 +1,592 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink; + +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.write.RequestConfiguration; +import org.apache.flink.connector.cassandra.sink.exception.CassandraFailureHandler; +import org.apache.flink.connector.cassandra.sink.exception.MaxRetriesExceededException; +import org.apache.flink.connector.cassandra.sink.writer.CassandraRecordWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.types.Row; +import org.apache.flink.util.function.ThrowingRunnable; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.datastax.driver.core.exceptions.InvalidQueryException; +import com.datastax.driver.core.exceptions.NoHostAvailableException; +import com.datastax.driver.core.exceptions.ReadTimeoutException; +import com.datastax.driver.core.exceptions.UnavailableException; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.concurrent.RejectedExecutionException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link CassandraSinkWriter}. */ +public class CassandraSinkWriterTest { + + @Mock private SinkWriterMetricGroup metricGroup; + private final TestCounter successCounter = new TestCounter(); + private final TestCounter failedCounter = new TestCounter(); + private final TestCounter retryCounter = new TestCounter(); + @Mock private Histogram latencyHistogram; + @Mock private OperatorIOMetricGroup ioMetricGroup; + private ClusterBuilder clusterBuilder; + + private CassandraFailureHandler failureHandler; + private CqlSinkConfig sinkConfig; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + + // Setup metric group mocking + when(metricGroup.getIOMetricGroup()).thenReturn(ioMetricGroup); + when(ioMetricGroup.getNumRecordsOutCounter()).thenReturn(successCounter); + when(metricGroup.getNumRecordsOutErrorsCounter()).thenReturn(failedCounter); + when(metricGroup.counter("retries")).thenReturn(retryCounter); + when(metricGroup.histogram(any(), any())).thenReturn(latencyHistogram); + + // Create a simple serializable cluster builder for testing + clusterBuilder = new SerializableClusterBuilder(); + + failureHandler = new CassandraFailureHandler(); + sinkConfig = CqlSinkConfig.forRow(); + } + + /** Simple test counter that tracks increments. */ + private static final class TestCounter implements Counter { + private long count = 0; + + @Override + public void inc() { + count++; + } + + @Override + public void inc(long n) { + count += n; + } + + @Override + public void dec() { + count--; + } + + @Override + public void dec(long n) { + count -= n; + } + + @Override + public long getCount() { + return count; + } + } + + /** Simple serializable cluster builder for testing. */ + private static final class SerializableClusterBuilder extends ClusterBuilder { + @Override + protected Cluster buildCluster(Cluster.Builder builder) { + return null; // Not needed for tests + } + } + + /** Direct mailbox executor that runs commands immediately for testing. */ + private static final class DirectMailbox implements MailboxExecutor { + @Override + public void execute( + ThrowingRunnable command, + String descriptionFormat, + Object... descriptionArgs) { + try { + command.run(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void yield() { + /* no-op */ + } + + @Override + public boolean tryYield() { + return false; + } + } + + /** Mailbox executor that always rejects execution for testing permit leak scenarios. */ + private static final class RejectingMailbox implements MailboxExecutor { + @Override + public void execute( + ThrowingRunnable command, + String descriptionFormat, + Object... descriptionArgs) { + throw new RejectedExecutionException("Test rejection"); + } + + @Override + public void yield() { + /* no-op */ + } + + @Override + public boolean tryYield() { + return false; + } + } + + /** Mock record writer that can be configured to return specific futures. */ + private static final class ControllableRecordWriter implements CassandraRecordWriter { + private ListenableFuture nextResult; + private int callCount = 0; + private ListenableFuture firstCallResult; + private ListenableFuture secondCallResult; + + void setNextResult(ListenableFuture future) { + this.nextResult = future; + } + + void setFirstThenSecondResult( + ListenableFuture first, ListenableFuture second) { + this.firstCallResult = first; + this.secondCallResult = second; + } + + @Override + public Session getSession() { + return mock(Session.class); + } + + @Override + public Statement prepareStatement(Row input) { + return mock(Statement.class); + } + + @Override + public ListenableFuture executeStatement(Statement statement) { + callCount++; + + if (firstCallResult != null && secondCallResult != null) { + if (callCount == 1) { + return firstCallResult; + } else if (callCount == 2) { + return secondCallResult; + } + } + + if (nextResult == null) { + throw new IllegalStateException("No result configured"); + } + return nextResult; + } + + @Override + public ListenableFuture write(Row record) { + return executeStatement(prepareStatement(record)); + } + + int getCallCount() { + return callCount; + } + + @Override + public void close() { + // no-op + } + } + + /** Creates a RequestConfiguration.Builder with tiny values for timeout tests. */ + private RequestConfiguration.Builder tinyRequestConfig() { + return RequestConfiguration.builder() + .setMaxConcurrentRequests(1) + .setMaxRetries(1) + .setMaxTimeout(Duration.ofMillis(10)); + } + + @Test + void basicTest() { + // Just verify the class loads and basic setup works + assertThat(failureHandler).isNotNull(); + assertThat(sinkConfig).isNotNull(); + assertThat(clusterBuilder).isNotNull(); + } + + @Test + void testExceptionClassification() { + // Based on actual CassandraFatalExceptionClassifier behavior: + // InvalidQueryException IS fatal (non-retriable) + assertThat(failureHandler.isFatal(new InvalidQueryException("test"))).isTrue(); + + // ReadTimeoutException and UnavailableException are NOT fatal (retriable) + ReadTimeoutException timeout = new ReadTimeoutException(null, null, 0, 1, false); + assertThat(failureHandler.isFatal(timeout)).isFalse(); + + UnavailableException unavailable = new UnavailableException(null, null, 1, 0); + assertThat(failureHandler.isFatal(unavailable)).isFalse(); + } + + @Test + void testRequestConfigurationBuilder() { + // Test that request configuration can be built with various parameters + RequestConfiguration config = + RequestConfiguration.builder() + .setMaxConcurrentRequests(5) + .setMaxRetries(3) + .setMaxTimeout(Duration.ofSeconds(30)) + .build(); + + assertThat(config.getMaxConcurrentRequests()).isEqualTo(5); + assertThat(config.getMaxRetries()).isEqualTo(3); + assertThat(config.getMaxTimeout()).isEqualTo(Duration.ofSeconds(30)); + } + + @Test + void testTinyRequestConfig() { + // Test the helper method for creating test configurations + RequestConfiguration config = tinyRequestConfig().build(); + + assertThat(config.getMaxConcurrentRequests()).isEqualTo(1); + assertThat(config.getMaxRetries()).isEqualTo(1); + assertThat(config.getMaxTimeout()).isEqualTo(Duration.ofMillis(10)); + } + + @Test + void testFailureHandlerCreation() { + // Test that failure handler can be created and basic methods work + CassandraFailureHandler handler = new CassandraFailureHandler(); + assertThat(handler).isNotNull(); + + // Test with various exception types + InvalidQueryException invalidQuery = new InvalidQueryException("Invalid CQL"); + assertThat(handler.isFatal(invalidQuery)).isTrue(); + + ReadTimeoutException timeout = new ReadTimeoutException(null, null, 0, 1, false); + assertThat(handler.isFatal(timeout)).isFalse(); + + RuntimeException generic = new RuntimeException("Generic error"); + assertThat(handler.isFatal(generic)).isFalse(); + } + + @Test + void testComponentCreation() { + // Test that all components can be created successfully + + // Sink configuration + CqlSinkConfig config = CqlSinkConfig.forRow(); + assertThat(config).isNotNull(); + + // Cluster builder + SerializableClusterBuilder builder = new SerializableClusterBuilder(); + assertThat(builder).isNotNull(); + assertThat(builder.buildCluster(null)).isNull(); // Designed for testing + + // Request configuration + RequestConfiguration requestConfig = + RequestConfiguration.builder() + .setMaxConcurrentRequests(5) + .setMaxTimeout(Duration.ofSeconds(30)) + .build(); + assertThat(requestConfig).isNotNull(); + + // Sink writer with various configurations + CassandraSinkWriter writer = + new CassandraSinkWriter<>( + requestConfig, + sinkConfig, + clusterBuilder, + failureHandler, + new DirectMailbox(), + metricGroup, + (cb, sc) -> null); + + assertThat(writer).isNotNull(); + } + + @Test + void testPermitAcquisitionTimeout() throws Exception { + // Setup: maxConcurrentRequests=1, tiny timeout, first write never completes + RequestConfiguration config = + RequestConfiguration.builder() + .setMaxConcurrentRequests(1) + .setMaxTimeout(Duration.ofMillis(500)) + .build(); + + ControllableRecordWriter recordWriter = new ControllableRecordWriter(); + SettableFuture neverCompletingFuture = SettableFuture.create(); + recordWriter.setNextResult(neverCompletingFuture); + + CassandraFailureHandler spyHandler = spy(failureHandler); + + CassandraSinkWriter writer = + new CassandraSinkWriter<>( + config, + sinkConfig, + clusterBuilder, + spyHandler, + new DirectMailbox(), + metricGroup, + (cb, sc) -> recordWriter); + + // First write should succeed in acquiring permit but not complete + writer.write( + Row.of(1, "test1"), + mock(org.apache.flink.api.connector.sink2.SinkWriter.Context.class)); + + // Second write should timeout trying to acquire permit + assertThatThrownBy( + () -> + writer.write( + Row.of(2, "test2"), + mock( + org.apache.flink.api.connector.sink2.SinkWriter + .Context.class))) + .isInstanceOf(IOException.class) + .hasMessageContaining("Failed to acquire"); + + // Verify failure handler was NOT called (timeout is not handled by failure handler) + verify(spyHandler, never()).onFailure(any()); + + // Verify permit leak by checking flush behavior - should timeout because first write + // never completed and still holds permit + assertThatThrownBy(() -> writer.flush(false)) + .isInstanceOf(IOException.class) + .hasMessageContaining("Unable to flush within timeout"); + } + + @Test + void testRetryThenSuccess() throws Exception { + // Setup: first call fails with retriable exception, second succeeds + RequestConfiguration config = + RequestConfiguration.builder().setMaxConcurrentRequests(2).setMaxRetries(2).build(); + + ControllableRecordWriter recordWriter = new ControllableRecordWriter(); + + // First call fails with retriable exception + SettableFuture failingFuture = SettableFuture.create(); + failingFuture.setException(new NoHostAvailableException(Collections.emptyMap())); + + // Second call (retry) succeeds + SettableFuture successFuture = SettableFuture.create(); + successFuture.set(mock(ResultSet.class)); + + recordWriter.setFirstThenSecondResult(failingFuture, successFuture); + + CassandraSinkWriter writer = + new CassandraSinkWriter<>( + config, + sinkConfig, + clusterBuilder, + failureHandler, + new DirectMailbox(), + metricGroup, + (cb, sc) -> recordWriter); + + // Write should succeed after retry + writer.write( + Row.of(1, "test"), + mock(org.apache.flink.api.connector.sink2.SinkWriter.Context.class)); + + // Verify retry occurred + assertThat(recordWriter.getCallCount()).isEqualTo(2); + assertThat(retryCounter.getCount()).isEqualTo(1); + + // Verify success metrics + assertThat(successCounter.getCount()).isEqualTo(1); + assertThat(failedCounter.getCount()).isEqualTo(0); + + writer.close(); + } + + @Test + void testMaxRetriesExhausted() throws Exception { + RequestConfiguration config = + RequestConfiguration.builder().setMaxConcurrentRequests(2).setMaxRetries(1).build(); + + ControllableRecordWriter recordWriter = new ControllableRecordWriter(); + SettableFuture failingFuture = SettableFuture.create(); + failingFuture.setException(new ReadTimeoutException(null, null, 1, 0, false)); + recordWriter.setNextResult(failingFuture); + + CassandraFailureHandler spyHandler = spy(failureHandler); + + CassandraSinkWriter writer = + new CassandraSinkWriter<>( + config, + sinkConfig, + clusterBuilder, + spyHandler, + new DirectMailbox(), + metricGroup, + (cb, sc) -> recordWriter); + + Row testRow = Row.of(1, "test"); + + // Write the element - this will submit the async operation + writer.write(testRow, mock(org.apache.flink.api.connector.sink2.SinkWriter.Context.class)); + + // Verify retries occurred (original + 1 retry = 2 total calls) + assertThat(recordWriter.getCallCount()).isEqualTo(2); + assertThat(retryCounter.getCount()).isEqualTo(1); + + // Verify failure handler was called with MaxRetriesExceededException + verify(spyHandler).onFailure(any(MaxRetriesExceededException.class)); + + // Verify failure metrics + assertThat(failedCounter.getCount()).isEqualTo(1); + assertThat(successCounter.getCount()).isEqualTo(0); + + writer.close(); + } + + @Test + void testImmediateFatalError() throws Exception { + // Setup: fail with fatal exception (no retries) + RequestConfiguration config = + RequestConfiguration.builder() + .setMaxConcurrentRequests(2) + .setMaxRetries(3) // Should not matter for fatal errors + .build(); + + ControllableRecordWriter recordWriter = new ControllableRecordWriter(); + SettableFuture failingFuture = SettableFuture.create(); + failingFuture.setException(new InvalidQueryException("Invalid table")); + recordWriter.setNextResult(failingFuture); + + CassandraFailureHandler spyHandler = spy(failureHandler); + + CassandraSinkWriter writer = + new CassandraSinkWriter<>( + config, + sinkConfig, + clusterBuilder, + spyHandler, + new DirectMailbox(), + metricGroup, + (cb, sc) -> recordWriter); + + // Write the element - this will submit the async operation + writer.write( + Row.of(1, "test"), + mock(org.apache.flink.api.connector.sink2.SinkWriter.Context.class)); + + // Verify NO retries occurred (only 1 call) + assertThat(recordWriter.getCallCount()).isEqualTo(1); + assertThat(retryCounter.getCount()).isEqualTo(0); + + // Verify failure handler was called with original fatal exception + verify(spyHandler).onFailure(any(InvalidQueryException.class)); + + // Verify failure metrics + assertThat(failedCounter.getCount()).isEqualTo(1); + assertThat(successCounter.getCount()).isEqualTo(0); + + writer.close(); + } + + @Test + void testFlushTimeout() throws Exception { + // Setup: in-flight request that never completes, small flush timeout + RequestConfiguration config = + RequestConfiguration.builder() + .setMaxConcurrentRequests(1) + .setFlushTimeout(Duration.ofMillis(500)) + .build(); + + ControllableRecordWriter recordWriter = new ControllableRecordWriter(); + SettableFuture neverCompletingFuture = SettableFuture.create(); + recordWriter.setNextResult(neverCompletingFuture); + + CassandraSinkWriter writer = + new CassandraSinkWriter<>( + config, + sinkConfig, + clusterBuilder, + failureHandler, + new DirectMailbox(), + metricGroup, + (cb, sc) -> recordWriter); + + // Start a write that will never complete + writer.write( + Row.of(1, "test"), + mock(org.apache.flink.api.connector.sink2.SinkWriter.Context.class)); + + // flush() should timeout + assertThatThrownBy(() -> writer.flush(false)) + .isInstanceOf(IOException.class) + .hasMessageContaining("Unable to flush within timeout") + .hasCauseInstanceOf(java.util.concurrent.TimeoutException.class); + + // Don't call close() as it will also timeout with in-flight requests + } + + @Test + void testExecutePendingTasksFastPath() throws Exception { + // Setup: no in-flight requests + RequestConfiguration config = + RequestConfiguration.builder().setMaxConcurrentRequests(2).build(); + + ControllableRecordWriter recordWriter = new ControllableRecordWriter(); + CassandraSinkWriter writer = + new CassandraSinkWriter<>( + config, + sinkConfig, + clusterBuilder, + failureHandler, + new DirectMailbox(), + metricGroup, + (cb, sc) -> recordWriter); + + // executePendingTasks should return immediately when no requests are in-flight + long startTime = System.nanoTime(); + writer.executePendingTasks(); + long duration = System.nanoTime() - startTime; + + // Should complete very quickly (less than 10ms) + assertThat(duration).isLessThan(10_000_000L); // 10ms in nanoseconds + + writer.close(); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/assembler/DynamicPlannerAssemblerTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/assembler/DynamicPlannerAssemblerTest.java new file mode 100644 index 0000000..6544eb9 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/assembler/DynamicPlannerAssemblerTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.assembler; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.SinkPluggable; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link DynamicPlannerAssembler}. */ +public class DynamicPlannerAssemblerTest { + + @Mock private CqlSinkConfig config; + @Mock private TableResolver tableResolver; + @Mock private ColumnValueResolver columnValueResolver; + @Mock private CqlClauseResolver clauseResolver; + @Mock private StatementCustomizer customizer; + + private DynamicPlannerAssembler assembler; + private SinkPluggable pluggable; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + assembler = new DynamicPlannerAssembler<>(config); + } + + @Test + void testAssembleWithInsertStrategy() { + // Setup + pluggable = + SinkPluggable.builder() + .withTableResolver(tableResolver) + .withColumnValueResolver(columnValueResolver) + .withCqlClauseResolver(clauseResolver) + .withStatementCustomizer(customizer) + .build(); + + when(config.getPluggable()).thenReturn(pluggable); + when(columnValueResolver.kind()).thenReturn(ColumnValueResolver.Kind.INSERT); + + // Execute + StatementPlanner planner = assembler.assemble(); + + // Verify + assertThat(planner).isNotNull(); + } + + @Test + void testAssembleWithUpdateStrategy() { + // Setup + pluggable = + SinkPluggable.builder() + .withTableResolver(tableResolver) + .withColumnValueResolver(columnValueResolver) + .withCqlClauseResolver(clauseResolver) + .withStatementCustomizer(customizer) + .build(); + + when(config.getPluggable()).thenReturn(pluggable); + when(columnValueResolver.kind()).thenReturn(ColumnValueResolver.Kind.UPDATE); + + // Execute + StatementPlanner planner = assembler.assemble(); + + // Verify + assertThat(planner).isNotNull(); + } + + @Test + void testAssembleValidationAndEdgeCases() { + // Test 1: Null optional components (clauseResolver and customizer) are allowed + pluggable = + SinkPluggable.builder() + .withTableResolver(tableResolver) + .withColumnValueResolver(columnValueResolver) + .withCqlClauseResolver(null) // Optional + .withStatementCustomizer(null) // Optional + .build(); + + when(config.getPluggable()).thenReturn(pluggable); + when(columnValueResolver.kind()).thenReturn(ColumnValueResolver.Kind.INSERT); + + StatementPlanner planner = assembler.assemble(); + assertThat(planner).isNotNull(); + + // Test 2: Null SinkPluggable throws exception + when(config.getPluggable()).thenReturn(null); + assertThatThrownBy(() -> assembler.assemble()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("SinkPluggable cannot be null in DYNAMIC mode"); + + // Test 3: Null TableResolver throws exception when building pluggable + assertThatThrownBy( + () -> + SinkPluggable.builder() + .withTableResolver(null) + .withColumnValueResolver(columnValueResolver) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("TableResolver cannot be null"); + + // Test 4: Null ColumnValueResolver throws exception when building pluggable + assertThatThrownBy( + () -> + SinkPluggable.builder() + .withTableResolver(tableResolver) + .withColumnValueResolver(null) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("ColumnValueResolver cannot be null"); + + // Test 5: Null config in constructor is allowed but fails on assemble + assertThatThrownBy(() -> new DynamicPlannerAssembler<>(null).assemble()) + .isInstanceOf(NullPointerException.class); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/assembler/StaticPlannerAssemblerTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/assembler/StaticPlannerAssemblerTest.java new file mode 100644 index 0000000..4424889 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/assembler/StaticPlannerAssemblerTest.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.assembler; + +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.RecordFormatType; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link StaticPlannerAssembler}. */ +public class StaticPlannerAssemblerTest { + + @Mock private CqlSinkConfig config; + private StaticPlannerAssembler assembler; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + assembler = new StaticPlannerAssembler<>(config); + } + + @Test + void testAssembleWithInsertAndUpdateQueries() { + // Test 1: INSERT query + String insertQuery = "INSERT INTO keyspace.table (id, name) VALUES (?, ?)"; + when(config.getQuery()).thenReturn(insertQuery); + when(config.getRecordFormatType()).thenReturn(RecordFormatType.POJO); + when(config.getIgnoreNullFields()).thenReturn(false); + + StatementPlanner insertPlanner = assembler.assemble(); + assertThat(insertPlanner).isNotNull(); + + // Test 2: UPDATE query + String updateQuery = "UPDATE keyspace.table SET name = ? WHERE id = ?"; + when(config.getQuery()).thenReturn(updateQuery); + + StatementPlanner updatePlanner = assembler.assemble(); + assertThat(updatePlanner).isNotNull(); + } + + @Test + void testAssembleWithIgnoreNullFields() { + // Test 1: INSERT query with ignoreNullFields enabled + String insertQuery = "INSERT INTO keyspace.table (id, name) VALUES (?, ?)"; + when(config.getQuery()).thenReturn(insertQuery); + when(config.getRecordFormatType()).thenReturn(RecordFormatType.POJO); + when(config.getIgnoreNullFields()).thenReturn(true); + + StatementPlanner insertPlanner = assembler.assemble(); + assertThat(insertPlanner).isNotNull(); + + // Test 2: UPDATE query with ignoreNullFields enabled + String updateQuery = "UPDATE keyspace.table SET name = ? WHERE id = ?"; + when(config.getQuery()).thenReturn(updateQuery); + + StatementPlanner updatePlanner = assembler.assemble(); + assertThat(updatePlanner).isNotNull(); + } + + @Test + void testAssembleWithRowType() { + // Setup for Row type + CqlSinkConfig rowConfig = + CqlSinkConfig.forRow().withQuery("INSERT INTO keyspace.table (id) VALUES (?)"); + + StaticPlannerAssembler rowAssembler = new StaticPlannerAssembler<>(rowConfig); + + // Execute + StatementPlanner planner = rowAssembler.assemble(); + + // Verify + assertThat(planner).isNotNull(); + } + + @Test + void testAssembleWithTupleType() { + // Setup for Tuple type + CqlSinkConfig tupleConfig = + CqlSinkConfig.forTuple() + .withQuery("UPDATE keyspace.table SET value = ? WHERE key = ?"); + + StaticPlannerAssembler tupleAssembler = new StaticPlannerAssembler<>(tupleConfig); + + // Execute + StatementPlanner planner = tupleAssembler.assemble(); + + // Verify + assertThat(planner).isNotNull(); + } + + @Test + void testAssembleWithCaseVariations() { + // Test 1: Lowercase insert + when(config.getQuery()).thenReturn("insert into ks.tbl (id) values (?)"); + when(config.getRecordFormatType()).thenReturn(RecordFormatType.POJO); + when(config.getIgnoreNullFields()).thenReturn(false); + + StatementPlanner planner1 = assembler.assemble(); + assertThat(planner1).isNotNull(); + + // Test 2: Mixed case update + when(config.getQuery()).thenReturn("Update KS.TBL set NAME = ? where ID = ?"); + StatementPlanner planner2 = assembler.assemble(); + assertThat(planner2).isNotNull(); + + // Test 3: With extra spaces + when(config.getQuery()).thenReturn(" INSERT INTO ks.tbl (id) VALUES (?) "); + StatementPlanner planner3 = assembler.assemble(); + assertThat(planner3).isNotNull(); + } + + @Test + void testAssembleValidationAndErrors() { + // Test 1: Empty query throws exception + when(config.getQuery()).thenReturn(""); + assertThatThrownBy(() -> assembler.assemble()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Query cannot be empty"); + + // Test 2: Null query throws exception + when(config.getQuery()).thenReturn(null); + assertThatThrownBy(() -> assembler.assemble()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Query cannot be empty"); + + // Test 3: SELECT query throws exception + when(config.getQuery()).thenReturn("SELECT * FROM keyspace.table"); + assertThatThrownBy(() -> assembler.assemble()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Static mode only supports INSERT and UPDATE queries"); + + // Test 4: DELETE query throws exception + when(config.getQuery()).thenReturn("DELETE FROM keyspace.table WHERE id = ?"); + assertThatThrownBy(() -> assembler.assemble()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Static mode only supports INSERT and UPDATE queries"); + + // Test 5: Invalid syntax in INSERT query + when(config.getQuery()).thenReturn("INSERT INTO"); + when(config.getRecordFormatType()).thenReturn(RecordFormatType.POJO); + assertThatThrownBy(() -> assembler.assemble()).isInstanceOf(IllegalArgumentException.class); + + // Test 6: Invalid syntax in UPDATE query + when(config.getQuery()).thenReturn("UPDATE table"); + assertThatThrownBy(() -> assembler.assemble()).isInstanceOf(IllegalArgumentException.class); + + // Test 7: Null config in constructor is allowed but fails on assemble + assertThatThrownBy(() -> new StaticPlannerAssembler<>(null).assemble()) + .isInstanceOf(NullPointerException.class); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/CqlSinkConfigTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/CqlSinkConfigTest.java new file mode 100644 index 0000000..22c3fc8 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/CqlSinkConfigTest.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config; + +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.connector.cassandra.sink.planner.SinkPluggable; +import org.apache.flink.connector.cassandra.sink.planner.core.resolution.ResolutionMode; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; + +/** Unit tests for {@link CqlSinkConfig}. */ +class CqlSinkConfigTest { + + private static SinkPluggable createValidPluggable() { + return SinkPluggable.builder() + .withTableResolver(mock(TableResolver.class)) + .withColumnValueResolver(mock(ColumnValueResolver.class)) + .build(); + } + + @Test + void testFactoryMethodsAndDefaults() { + // Test Row config + CqlSinkConfig rowConfig = CqlSinkConfig.forRow(); + assertThat(rowConfig.getRecordFormatType()).isEqualTo(RecordFormatType.ROW); + assertThat(rowConfig.getQuery()).isNull(); + assertThat(rowConfig.getPluggable()).isNull(); + assertThat(rowConfig.getIgnoreNullFields()).isFalse(); + assertThat(rowConfig.getResolutionMode()).isEqualTo(ResolutionMode.UNSET); + + // Test Tuple config + CqlSinkConfig tupleConfig = CqlSinkConfig.forTuple(); + assertThat(tupleConfig.getRecordFormatType()).isEqualTo(RecordFormatType.TUPLE); + assertThat(tupleConfig.getQuery()).isNull(); + assertThat(tupleConfig.getPluggable()).isNull(); + assertThat(tupleConfig.getIgnoreNullFields()).isFalse(); + assertThat(tupleConfig.getResolutionMode()).isEqualTo(ResolutionMode.UNSET); + + // Test Scala Product config + CqlSinkConfig scalaConfig = CqlSinkConfig.forScalaProduct(); + assertThat(scalaConfig.getRecordFormatType()).isEqualTo(RecordFormatType.SCALA_PRODUCT); + assertThat(scalaConfig.getQuery()).isNull(); + assertThat(scalaConfig.getPluggable()).isNull(); + assertThat(scalaConfig.getIgnoreNullFields()).isFalse(); + assertThat(scalaConfig.getResolutionMode()).isEqualTo(ResolutionMode.UNSET); + } + + @Test + void testStaticModeWithQuery() { + CqlSinkConfig config = + CqlSinkConfig.forRow() + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + + assertThat(config.getQuery()) + .isEqualTo("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + assertThat(config.getResolutionMode()).isEqualTo(ResolutionMode.STATIC); + assertThat(config.getRecordFormatType()).isEqualTo(RecordFormatType.ROW); + assertThat(config.getPluggable()).isNull(); + } + + @Test + void testDynamicModeWithPluggable() { + SinkPluggable pluggable = createValidPluggable(); + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + assertThat(config.getPluggable()).isEqualTo(pluggable); + assertThat(config.getResolutionMode()).isEqualTo(ResolutionMode.DYNAMIC); + assertThat(config.getQuery()).isNull(); + } + + @Test + void testModeConflictsAndValidation() { + // Test ignoreNullFields fails in DYNAMIC mode + SinkPluggable pluggable = createValidPluggable(); + CqlSinkConfig dynamicConfig = CqlSinkConfig.forRow().withPluggable(pluggable); + + assertThatThrownBy(() -> dynamicConfig.withIgnoreNullFields(true)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("ignoreNullFields is not supported in DYNAMIC mode") + .hasMessageContaining("control null/unset behavior via StatementCustomizer"); + + // Test query after pluggable fails + assertThatThrownBy( + () -> dynamicConfig.withQuery("INSERT INTO keyspace.table (id) VALUES (?)")) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Cannot set query when pluggables are already configured") + .hasMessageContaining("STATIC and DYNAMIC are mutually exclusive"); + + // Test pluggable after query fails + CqlSinkConfig staticConfig = + CqlSinkConfig.forRow().withQuery("INSERT INTO keyspace.table (id) VALUES (?)"); + + assertThatThrownBy(() -> staticConfig.withPluggable(pluggable)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Cannot set pluggables when a query is already configured") + .hasMessageContaining("STATIC and DYNAMIC are mutually exclusive"); + + // Test query validation + CqlSinkConfig config = CqlSinkConfig.forRow(); + assertThatThrownBy(() -> config.withQuery(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("query must not be blank"); + + assertThatThrownBy(() -> config.withQuery("")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("query must not be blank"); + + assertThatThrownBy(() -> config.withQuery(" ")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("query must not be blank"); + + // Test pluggable validation + assertThatThrownBy(() -> CqlSinkConfig.forRow().withPluggable(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("pluggable cannot be null"); + } + + @Test + void testResolutionModeDerivation() { + // Test with Row and STATIC mode + CqlSinkConfig rowConfig = CqlSinkConfig.forRow(); + assertThat(rowConfig.getResolutionMode()).isEqualTo(ResolutionMode.UNSET); + + rowConfig.withQuery("INSERT INTO keyspace.table (id) VALUES (?)"); + assertThat(rowConfig.getResolutionMode()).isEqualTo(ResolutionMode.STATIC); + + // Test with Tuple and DYNAMIC mode + CqlSinkConfig tupleConfig = CqlSinkConfig.forTuple(); + assertThat(tupleConfig.getResolutionMode()).isEqualTo(ResolutionMode.UNSET); + + @SuppressWarnings("unchecked") + SinkPluggable tuplePluggable = + SinkPluggable.builder() + .withTableResolver(mock(TableResolver.class)) + .withColumnValueResolver(mock(ColumnValueResolver.class)) + .build(); + + tupleConfig.withPluggable(tuplePluggable); + assertThat(tupleConfig.getResolutionMode()).isEqualTo(ResolutionMode.DYNAMIC); + assertThat(tupleConfig.getPluggable()).isEqualTo(tuplePluggable); + } + + @Test + void testAPIChaining() { + // STATIC mode chaining + CqlSinkConfig staticConfig = + CqlSinkConfig.forRow() + .withIgnoreNullFields(true) + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + + assertThat(staticConfig.getIgnoreNullFields()).isTrue(); + assertThat(staticConfig.getQuery()) + .isEqualTo("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + assertThat(staticConfig.getResolutionMode()).isEqualTo(ResolutionMode.STATIC); + + // DYNAMIC mode chaining + SinkPluggable pluggable = createValidPluggable(); + CqlSinkConfig dynamicConfig = CqlSinkConfig.forRow().withPluggable(pluggable); + + assertThat(dynamicConfig.getPluggable()).isEqualTo(pluggable); + assertThat(dynamicConfig.getResolutionMode()).isEqualTo(ResolutionMode.DYNAMIC); + } + + @Test + void testSinkPluggableBuilderValidation() { + // Test that SinkPluggable.builder() requires TableResolver and ColumnValueResolver + assertThatThrownBy(() -> SinkPluggable.builder().build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("TableResolver cannot be null"); + + // Test with only TableResolver + assertThatThrownBy( + () -> + SinkPluggable.builder() + .withTableResolver(mock(TableResolver.class)) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("ColumnValueResolver cannot be null"); + + // Test with only ColumnValueResolver + assertThatThrownBy( + () -> + SinkPluggable.builder() + .withColumnValueResolver(mock(ColumnValueResolver.class)) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("TableResolver cannot be null"); + + // Test successful build with both required resolvers + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(mock(TableResolver.class)) + .withColumnValueResolver(mock(ColumnValueResolver.class)) + .build(); + + assertThat(pluggable).isNotNull(); + assertThat(pluggable.getTableResolver()).isNotNull(); + assertThat(pluggable.getColumnValueResolver()).isNotNull(); + assertThat(pluggable.getCqlClauseResolver()).isNull(); + assertThat(pluggable.getStatementCustomizer()).isNull(); + } + + @Test + void testSerializationWithStaticMode() throws Exception { + CqlSinkConfig original = + CqlSinkConfig.forRow() + .withIgnoreNullFields(true) + .withQuery("INSERT INTO keyspace.table (id, name) VALUES (?, ?)"); + + // Serialize + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + oos.writeObject(original); + oos.close(); + + // Deserialize + ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + ObjectInputStream ois = new ObjectInputStream(bais); + @SuppressWarnings("unchecked") + CqlSinkConfig deserialized = (CqlSinkConfig) ois.readObject(); + ois.close(); + + // Verify + assertThat(deserialized.getQuery()).isEqualTo(original.getQuery()); + assertThat(deserialized.getIgnoreNullFields()).isEqualTo(original.getIgnoreNullFields()); + assertThat(deserialized.getResolutionMode()).isEqualTo(ResolutionMode.STATIC); + assertThat(deserialized.getRecordFormatType()).isEqualTo(RecordFormatType.ROW); + } + + @Test + void testSerializationWithDynamicMode() throws Exception { + SinkPluggable pluggable = createValidPluggable(); + + CqlSinkConfig original = CqlSinkConfig.forRow().withPluggable(pluggable); + + // Serialize + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + oos.writeObject(original); + oos.close(); + + // Deserialize + ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + ObjectInputStream ois = new ObjectInputStream(bais); + @SuppressWarnings("unchecked") + CqlSinkConfig deserialized = (CqlSinkConfig) ois.readObject(); + ois.close(); + + // Verify + assertThat(deserialized.getPluggable()).isNotNull(); + assertThat(deserialized.getIgnoreNullFields()).isFalse(); + assertThat(deserialized.getResolutionMode()).isEqualTo(ResolutionMode.DYNAMIC); + assertThat(deserialized.getRecordFormatType()).isEqualTo(RecordFormatType.ROW); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/PojoSinkConfigTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/PojoSinkConfigTest.java new file mode 100644 index 0000000..cd4681d --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/PojoSinkConfigTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config; + +import org.apache.flink.streaming.connectors.cassandra.SimpleMapperOptions; + +import com.datastax.driver.mapping.annotations.Column; +import com.datastax.driver.mapping.annotations.PartitionKey; +import com.datastax.driver.mapping.annotations.Table; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Unit tests for {@link PojoSinkConfig}. */ +class PojoSinkConfigTest { + + @Table(keyspace = "test_ks", name = "test_table") + static class TestPojo { + @PartitionKey + @Column(name = "id") + private String id; + + @Column(name = "name") + private String name; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + } + + @Test + void testPojoConfigurationWithoutMapperOptions() { + PojoSinkConfig config = + new PojoSinkConfig<>(TestPojo.class, "test_keyspace", null); + + assertThat(config.getPojoClass()).isEqualTo(TestPojo.class); + assertThat(config.getKeyspace()).isEqualTo("test_keyspace"); + assertThat(config.getMapperOptions()).isNull(); + assertThat(config.getRecordFormatType()).isEqualTo(RecordFormatType.POJO); + } + + @Test + void testPojoConfigurationWithMapperOptions() { + SimpleMapperOptions options = new SimpleMapperOptions().ttl(3600); + + PojoSinkConfig config = new PojoSinkConfig<>(TestPojo.class, "test_ks", options); + + assertThat(config.getPojoClass()).isEqualTo(TestPojo.class); + assertThat(config.getKeyspace()).isEqualTo("test_ks"); + assertThat(config.getMapperOptions()).isEqualTo(options); + assertThat(config.getRecordFormatType()).isEqualTo(RecordFormatType.POJO); + } + + @Test + void testConstructorValidation() { + // Null POJO class + assertThatThrownBy(() -> new PojoSinkConfig(null, "test_ks", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("pojoClass must not be null"); + + // Null keyspace + assertThatThrownBy(() -> new PojoSinkConfig<>(TestPojo.class, null, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Keyspace cannot be empty"); + + // Empty keyspace + assertThatThrownBy(() -> new PojoSinkConfig<>(TestPojo.class, "", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Keyspace cannot be empty"); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/write/RequestConfigurationTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/write/RequestConfigurationTest.java new file mode 100644 index 0000000..bdb3d02 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/config/write/RequestConfigurationTest.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.config.write; + +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.time.Duration; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Unit tests for {@link RequestConfiguration}. */ +class RequestConfigurationTest { + + @Test + void testDefaultValues() { + RequestConfiguration config = RequestConfiguration.builder().build(); + + // Verify all defaults match constants + assertThat(config.getMaxConcurrentRequests()) + .isEqualTo(RequestConfiguration.DEFAULT_MAX_CONCURRENT_REQUESTS) + .isEqualTo(Integer.MAX_VALUE); + assertThat(config.getMaxRetries()) + .isEqualTo(RequestConfiguration.DEFAULT_MAX_RETRIES) + .isEqualTo(0); + assertThat(config.getMaxTimeout()) + .isEqualTo(RequestConfiguration.DEFAULT_MAX_TIMEOUT) + .isEqualTo(Duration.ofMinutes(1)); + assertThat(config.getFlushTimeout()) + .isEqualTo(RequestConfiguration.DEFAULT_FLUSH_TIMEOUT) + .isEqualTo(Duration.ofSeconds(30)); + } + + @Test + void testCustomValues() { + RequestConfiguration config = + RequestConfiguration.builder() + .setMaxTimeout(Duration.ofSeconds(5)) + .setMaxRetries(5) + .setMaxConcurrentRequests(500) + .setFlushTimeout(Duration.ofSeconds(10)) + .build(); + + assertThat(config.getMaxTimeout()).isEqualTo(Duration.ofSeconds(5)); + assertThat(config.getMaxRetries()).isEqualTo(5); + assertThat(config.getMaxConcurrentRequests()).isEqualTo(500); + assertThat(config.getFlushTimeout()).isEqualTo(Duration.ofSeconds(10)); + } + + @Test + void testBuilderValidation() { + // Test maxConcurrentRequests validation + assertThatThrownBy(() -> RequestConfiguration.builder().setMaxConcurrentRequests(0).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("maxConcurrentRequests must be"); + + assertThatThrownBy( + () -> RequestConfiguration.builder().setMaxConcurrentRequests(-1).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("maxConcurrentRequests must be"); + + // Test maxTimeout validation + assertThatThrownBy( + () -> RequestConfiguration.builder().setMaxTimeout(Duration.ZERO).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("maxTimeout must be non-null and positive"); + + assertThatThrownBy( + () -> + RequestConfiguration.builder() + .setMaxTimeout(Duration.ofSeconds(-1)) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("maxTimeout must be non-null and positive"); + + assertThatThrownBy(() -> RequestConfiguration.builder().setMaxTimeout(null).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("maxTimeout must be non-null and positive"); + + // Test maxRetries validation + assertThatThrownBy(() -> RequestConfiguration.builder().setMaxRetries(-1).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("maxRetries must be"); + + // Zero retries should be allowed (means no retries) + RequestConfiguration config = RequestConfiguration.builder().setMaxRetries(0).build(); + assertThat(config.getMaxRetries()).isEqualTo(0); + } + + @Test + void testCombinedSetters() { + // Test if there are any convenience methods that set multiple values + RequestConfiguration config = + RequestConfiguration.builder() + .setMaxTimeout(Duration.ofMillis(1500)) + .setMaxRetries(2) + .setMaxConcurrentRequests(100) + .build(); + + assertThat(config.getMaxTimeout()).isEqualTo(Duration.ofMillis(1500)); + assertThat(config.getMaxRetries()).isEqualTo(2); + assertThat(config.getMaxConcurrentRequests()).isEqualTo(100); + } + + @Test + void testSerialization() throws Exception { + RequestConfiguration original = + RequestConfiguration.builder() + .setMaxTimeout(Duration.ofSeconds(10)) + .setMaxRetries(7) + .setMaxConcurrentRequests(2000) + .setFlushTimeout(Duration.ofSeconds(45)) + .build(); + + // Serialize + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + oos.writeObject(original); + oos.close(); + + // Deserialize + ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + ObjectInputStream ois = new ObjectInputStream(bais); + RequestConfiguration deserialized = (RequestConfiguration) ois.readObject(); + ois.close(); + + // Verify all fields preserved + assertThat(deserialized.getMaxTimeout()).isEqualTo(original.getMaxTimeout()); + assertThat(deserialized.getMaxRetries()).isEqualTo(original.getMaxRetries()); + assertThat(deserialized.getMaxConcurrentRequests()) + .isEqualTo(original.getMaxConcurrentRequests()); + assertThat(deserialized.getFlushTimeout()).isEqualTo(original.getFlushTimeout()); + } + + @Test + void testRejectsNonPositiveFlushTimeout() { + assertThatThrownBy( + () -> RequestConfiguration.builder().setFlushTimeout(Duration.ZERO).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("flushTimeout must be non-null and positive"); + + assertThatThrownBy( + () -> + RequestConfiguration.builder() + .setFlushTimeout(Duration.ofSeconds(-1)) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("flushTimeout must be non-null and positive"); + + assertThatThrownBy(() -> RequestConfiguration.builder().setFlushTimeout(null).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("flushTimeout must be non-null and positive"); + } + + @Test + void testFlushTimeoutAndMaxTimeoutAreIndependent() { + // Test that flushTimeout and maxTimeout are independent + RequestConfiguration config1 = + RequestConfiguration.builder() + .setMaxTimeout(Duration.ofSeconds(5)) + .setFlushTimeout(Duration.ofMinutes(2)) + .build(); + + assertThat(config1.getMaxTimeout()).isEqualTo(Duration.ofSeconds(5)); + assertThat(config1.getFlushTimeout()).isEqualTo(Duration.ofMinutes(2)); + + // Opposite configuration + RequestConfiguration config2 = + RequestConfiguration.builder() + .setMaxTimeout(Duration.ofMinutes(10)) + .setFlushTimeout(Duration.ofSeconds(1)) + .build(); + + assertThat(config2.getMaxTimeout()).isEqualTo(Duration.ofMinutes(10)); + assertThat(config2.getFlushTimeout()).isEqualTo(Duration.ofSeconds(1)); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/StatementPlannerFactoryTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/StatementPlannerFactoryTest.java new file mode 100644 index 0000000..9ac4ca0 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/StatementPlannerFactoryTest.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.RecordFormatType; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlannerFactory; +import org.apache.flink.connector.cassandra.sink.planner.resolver.FixedColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.FixedTableResolver; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Unit tests for {@link StatementPlannerFactory}. */ +public class StatementPlannerFactoryTest { + + @Test + void testCorrectAssemblerChosenForDynamicMode() { + // Create config for DYNAMIC mode with SinkPluggable + SinkPluggable pluggable = + SinkPluggable.builder() + .withTableResolver(new FixedTableResolver<>("test_ks", "test_table")) + .withColumnValueResolver( + new FixedColumnValueResolver<>( + RecordFormatType.ROW, Arrays.asList("id", "name"))) + .build(); + + CqlSinkConfig config = CqlSinkConfig.forRow().withPluggable(pluggable); + + // Create planner - should use DynamicAssembler + StatementPlanner planner = StatementPlannerFactory.create(config); + + // Verify planner was created (DynamicAssembler was used) + assertThat(planner).isNotNull(); + } + + @Test + void testCorrectAssemblerChosenForStaticMode() { + // Create config for STATIC mode + CqlSinkConfig config = + CqlSinkConfig.forRow().withQuery("INSERT INTO ks.tbl (id) VALUES (?)"); + + // Create planner - should use StaticAssembler + StatementPlanner planner = StatementPlannerFactory.create(config); + + // Verify planner was created (StaticAssembler was used) + assertThat(planner).isNotNull(); + } + + @Test + void testUnsetModeThrowsException() { + // Create config with neither query nor pluggable (UNSET mode) + CqlSinkConfig config = CqlSinkConfig.forRow(); + + // Should throw exception for UNSET mode + assertThatThrownBy(() -> StatementPlannerFactory.create(config)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid ResolutionMode UNSET"); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/StatementPlannerTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/StatementPlannerTest.java new file mode 100644 index 0000000..c21aa69 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/StatementPlannerTest.java @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; +import org.apache.flink.connector.cassandra.sink.planner.core.customization.StatementCustomizer; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.core.strategy.PlannerStrategy; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ColumnValueResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableResolver; + +import com.datastax.driver.core.BoundStatement; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.datastax.driver.core.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.Update; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link StatementPlanner}. */ +public class StatementPlannerTest { + + @Mock private Session session; + @Mock private PreparedStatement preparedStatement; + @Mock private BoundStatement boundStatement; + @Mock private TableResolver tableResolver; + @Mock private ColumnValueResolver columnValueResolver; + @Mock private PlannerStrategy strategy; + @Mock private StatementCustomizer customizer; + @Mock private CqlClauseResolver clauseResolver; + @Mock private CqlSinkConfig config; + // PreparedStatementCache is now internal to StatementPlanner + + private StatementPlanner planner; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + } + + @Test + void testDynamicInsertHappyPath() { + // Setup + String record = "test-record"; + TableRef tableRef = new TableRef("keyspace", "table"); + ResolvedWrite resolvedWrite = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + PlannerStrategy.QueryWithBindings queryWithBindings = + new PlannerStrategy.QueryWithBindings( + "INSERT INTO keyspace.table (id, name) VALUES (?, ?)", + ClauseBindings.empty()); + + when(tableResolver.resolve(record)).thenReturn(tableRef); + when(columnValueResolver.resolve(record)).thenReturn(resolvedWrite); + when(strategy.getQueryWithBindings(tableRef, config, clauseResolver, resolvedWrite, record)) + .thenReturn(queryWithBindings); + when(strategy.getOrderedBindValues(resolvedWrite, ClauseBindings.empty())) + .thenReturn(new Object[] {1, "Alice"}); + when(session.prepare(anyString())).thenReturn(preparedStatement); + when(preparedStatement.bind(any())).thenReturn(boundStatement); + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, clauseResolver, customizer); + Statement result = planner.plan(record, session, config); + // Verify + assertThat(result).isSameAs(boundStatement); + verify(tableResolver).resolve(record); + verify(columnValueResolver).resolve(record); + verify(strategy) + .getQueryWithBindings(tableRef, config, clauseResolver, resolvedWrite, record); + verify(session).prepare(anyString()); + verify(preparedStatement).bind(new Object[] {1, "Alice"}); // Values in correct order + verify(customizer).apply(boundStatement, record); + } + + @Test + void testDynamicUpdateWithClauseModifiers() { + // Setup + String record = "test-record"; + TableRef tableRef = new TableRef("keyspace", "table"); + ResolvedWrite resolvedWrite = + ResolvedWrite.update( + Arrays.asList("name", "email"), + new Object[] {"Bob", "bob@example.com"}, + Collections.singletonList("id"), + new Object[] {123}); + ClauseBindings clauseBindings = + new ClauseBindings(new Object[] {3600L}, new Object[0]); // TTL value + PlannerStrategy.QueryWithBindings queryWithBindings = + new PlannerStrategy.QueryWithBindings( + "UPDATE keyspace.table SET name = ?, email = ? WHERE id = ? USING TTL ?", + clauseBindings); + + when(tableResolver.resolve(record)).thenReturn(tableRef); + when(columnValueResolver.resolve(record)).thenReturn(resolvedWrite); + // Mock both methods since we don't know which will be called + when(clauseResolver.applyTo(any(Insert.class), eq(record))).thenReturn(clauseBindings); + when(clauseResolver.applyTo(any(Update.class), eq(record))).thenReturn(clauseBindings); + when(strategy.getQueryWithBindings(tableRef, config, clauseResolver, resolvedWrite, record)) + .thenReturn(queryWithBindings); + when(strategy.getOrderedBindValues(resolvedWrite, clauseBindings)) + .thenReturn(new Object[] {"Bob", "bob@example.com", 123, 3600L}); + when(session.prepare(anyString())).thenReturn(preparedStatement); + when(preparedStatement.bind(any())).thenReturn(boundStatement); + // customizer now returns void + + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, clauseResolver, customizer); + + // Execute + Statement result = planner.plan(record, session, config); + + // Verify bind order + verify(preparedStatement).bind(new Object[] {"Bob", "bob@example.com", 123, 3600L}); + verify(customizer).apply(boundStatement, record); + } + + @Test + void testCacheReuseOnSecondCall() { + // Setup + String record = "test-record"; + TableRef tableRef = new TableRef("keyspace", "table"); + ResolvedWrite resolvedWrite = + ResolvedWrite.insert(Collections.singletonList("id"), new Object[] {1}); + PlannerStrategy.QueryWithBindings queryWithBindings = + new PlannerStrategy.QueryWithBindings( + "INSERT INTO keyspace.table (id) VALUES (?)", ClauseBindings.empty()); + + when(tableResolver.resolve(record)).thenReturn(tableRef); + when(columnValueResolver.resolve(record)).thenReturn(resolvedWrite); + when(strategy.getQueryWithBindings(tableRef, config, clauseResolver, resolvedWrite, record)) + .thenReturn(queryWithBindings); + when(strategy.getOrderedBindValues(resolvedWrite, ClauseBindings.empty())) + .thenReturn(new Object[] {1}); + when(session.prepare(anyString())).thenReturn(preparedStatement); + when(preparedStatement.bind(any())).thenReturn(boundStatement); + // customizer now returns void + + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, clauseResolver, customizer); + + // Execute twice + planner.plan(record, session, config); + planner.plan(record, session, config); + + // Verify cache is used (prepare should only be called once due to internal caching) + verify(session, times(1)).prepare(anyString()); + verify(strategy, times(2)) + .getQueryWithBindings(tableRef, config, clauseResolver, resolvedWrite, record); + } + + @Test + void testStaticModeIgnoresClauseResolver() { + // Setup for static mode + String record = "test-record"; + TableRef tableRef = new TableRef("keyspace", "table"); + ResolvedWrite resolvedWrite = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + PlannerStrategy.QueryWithBindings queryWithBindings = + new PlannerStrategy.QueryWithBindings( + "INSERT INTO keyspace.table (id, name) VALUES (?, ?)", + ClauseBindings.empty()); + + when(tableResolver.resolve(record)).thenReturn(tableRef); + when(columnValueResolver.resolve(record)).thenReturn(resolvedWrite); + // Note: null passed for clauseResolver in static mode + when(strategy.getQueryWithBindings( + eq(tableRef), eq(config), any(), eq(resolvedWrite), eq(record))) + .thenReturn(queryWithBindings); + when(strategy.getOrderedBindValues(resolvedWrite, ClauseBindings.empty())) + .thenReturn(new Object[] {1, "Alice"}); + when(session.prepare(anyString())).thenReturn(preparedStatement); + when(preparedStatement.bind(any())).thenReturn(boundStatement); + // customizer now returns void + + // Use null clauseResolver for static mode + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, null, customizer); + + // Execute + Statement result = planner.plan(record, session, config); + + // Verify clauseResolver was not used (any() matcher allows NoOpClauseResolver) + verify(strategy) + .getQueryWithBindings( + eq(tableRef), eq(config), any(), eq(resolvedWrite), eq(record)); + verifyNoInteractions(clauseResolver); + assertThat(result).isSameAs(boundStatement); + } + + @Test + void testExceptionPropagationFromComponents() { + // Test 1: TableResolver exception propagates + RuntimeException tableError = new RuntimeException("Table resolution failed"); + when(tableResolver.resolve(any())).thenThrow(tableError); + + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, clauseResolver, customizer); + + assertThatThrownBy(() -> planner.plan("record", session, config)).isSameAs(tableError); + + // Reset mocks for next test + MockitoAnnotations.openMocks(this); + + // Test 2: ColumnValueResolver exception propagates + when(tableResolver.resolve(any())).thenReturn(new TableRef("ks", "tbl")); + RuntimeException columnError = new RuntimeException("Column resolution failed"); + when(columnValueResolver.resolve(any())).thenThrow(columnError); + + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, clauseResolver, customizer); + + assertThatThrownBy(() -> planner.plan("record", session, config)).isSameAs(columnError); + + // Reset mocks for next test + MockitoAnnotations.openMocks(this); + + // Test 3: Strategy exception propagates + when(tableResolver.resolve(any())).thenReturn(new TableRef("ks", "tbl")); + when(columnValueResolver.resolve(any())) + .thenReturn( + ResolvedWrite.insert(Collections.singletonList("id"), new Object[] {1})); + IllegalArgumentException strategyError = + new IllegalArgumentException("Wrong operation kind"); + when(strategy.getQueryWithBindings(any(), any(), any(), any(), any())) + .thenThrow(strategyError); + + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, clauseResolver, customizer); + + assertThatThrownBy(() -> planner.plan("record", session, config)).isSameAs(strategyError); + } + + @Test + void testSessionPrepareFailurePropagates() { + // Setup + String record = "test-record"; + TableRef tableRef = new TableRef("keyspace", "table"); + ResolvedWrite resolvedWrite = + ResolvedWrite.insert(Collections.singletonList("id"), new Object[] {1}); + PlannerStrategy.QueryWithBindings queryWithBindings = + new PlannerStrategy.QueryWithBindings( + "INSERT INTO keyspace.table (id) VALUES (?)", ClauseBindings.empty()); + + when(tableResolver.resolve(record)).thenReturn(tableRef); + when(columnValueResolver.resolve(record)).thenReturn(resolvedWrite); + when(strategy.getQueryWithBindings(tableRef, config, clauseResolver, resolvedWrite, record)) + .thenReturn(queryWithBindings); + + RuntimeException prepareError = new RuntimeException("Session prepare failed"); + when(session.prepare(anyString())).thenThrow(prepareError); + + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, clauseResolver, customizer); + + assertThatThrownBy(() -> planner.plan(record, session, config)) + .isInstanceOf(com.google.common.util.concurrent.UncheckedExecutionException.class) + .hasCauseInstanceOf(RuntimeException.class) + .hasRootCauseMessage("Session prepare failed"); + } + + @Test + void testNullCustomizerAllowed() { + // Setup with null customizer + String record = "test-record"; + TableRef tableRef = new TableRef("keyspace", "table"); + ResolvedWrite resolvedWrite = + ResolvedWrite.insert(Collections.singletonList("id"), new Object[] {1}); + PlannerStrategy.QueryWithBindings queryWithBindings = + new PlannerStrategy.QueryWithBindings( + "INSERT INTO keyspace.table (id) VALUES (?)", ClauseBindings.empty()); + + when(tableResolver.resolve(record)).thenReturn(tableRef); + when(columnValueResolver.resolve(record)).thenReturn(resolvedWrite); + when(strategy.getQueryWithBindings(tableRef, config, clauseResolver, resolvedWrite, record)) + .thenReturn(queryWithBindings); + when(strategy.getOrderedBindValues(resolvedWrite, ClauseBindings.empty())) + .thenReturn(new Object[] {1}); + when(session.prepare(anyString())).thenReturn(preparedStatement); + when(preparedStatement.bind(any())).thenReturn(boundStatement); + + planner = + new StatementPlanner<>( + tableResolver, columnValueResolver, strategy, clauseResolver, null); + + // Execute + Statement result = planner.plan(record, session, config); + + // Verify customizer not used + assertThat(result).isSameAs(boundStatement); + verifyNoInteractions(customizer); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/cache/PreparedStatementCacheTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/cache/PreparedStatementCacheTest.java new file mode 100644 index 0000000..009d383 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/cache/PreparedStatementCacheTest.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.cache; + +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Session; +import com.google.common.cache.CacheStats; +import com.google.common.util.concurrent.UncheckedExecutionException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link PreparedStatementCache}. */ +public class PreparedStatementCacheTest { + + @Mock private Session session; + @Mock private PreparedStatement preparedStatement1; + @Mock private PreparedStatement preparedStatement2; + + private PreparedStatementCache cache; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + cache = new PreparedStatementCache(); + } + + @Test + void testReturnsSameInstanceForSameKey() { + String insertQuery = "INSERT INTO ks.table (id, name) VALUES (?, ?)"; + when(session.prepare(insertQuery)).thenReturn(preparedStatement1); + + PreparedStatement result1 = cache.getOrPrepare(session, insertQuery); + PreparedStatement result2 = cache.getOrPrepare(session, insertQuery); + + assertThat(result1).isSameAs(result2); + verify(session, times(1)).prepare(insertQuery); + } + + @Test + void testDifferentKeysReturnDifferentStatements() { + String insertQuery1 = "INSERT INTO ks.table1 (id) VALUES (?)"; + String insertQuery2 = "INSERT INTO ks.table2 (id, name) VALUES (?, ?)"; + when(session.prepare(insertQuery1)).thenReturn(preparedStatement1); + when(session.prepare(insertQuery2)).thenReturn(preparedStatement2); + + PreparedStatement result1 = cache.getOrPrepare(session, insertQuery1); + PreparedStatement result2 = cache.getOrPrepare(session, insertQuery2); + + assertThat(result1).isNotSameAs(result2); + assertThat(result1).isSameAs(preparedStatement1); + assertThat(result2).isSameAs(preparedStatement2); + } + + @Test + void testExceptionsArePropagatedWithWrapping() { + // Test session.prepare() exceptions + RuntimeException sessionException = new RuntimeException("Session prepare failed"); + when(session.prepare(anyString())).thenThrow(sessionException); + + assertThatThrownBy( + () -> cache.getOrPrepare(session, "INSERT INTO ks.table (id) VALUES (?)")) + .isInstanceOf(UncheckedExecutionException.class) + .hasCauseInstanceOf(RuntimeException.class) + .hasRootCauseMessage("Session prepare failed"); + } + + @Test + void testThreadSafety() throws InterruptedException { + String insertQuery = "INSERT INTO ks.table (id) VALUES (?)"; + when(session.prepare(insertQuery)).thenReturn(preparedStatement1); + + int threadCount = 10; + CountDownLatch latch = new CountDownLatch(threadCount); + ExecutorService executor = Executors.newFixedThreadPool(threadCount); + CompletableFuture[] futures = new CompletableFuture[threadCount]; + + for (int i = 0; i < threadCount; i++) { + futures[i] = + CompletableFuture.supplyAsync( + () -> { + try { + return cache.getOrPrepare(session, insertQuery); + } finally { + latch.countDown(); + } + }, + executor); + } + + latch.await(); + executor.shutdown(); + + PreparedStatement firstResult = futures[0].join(); + for (CompletableFuture future : futures) { + assertThat(future.join()).isSameAs(firstResult); + } + + verify(session, times(1)).prepare(insertQuery); + } + + @Test + void testStatisticsTracking() { + // Create a fresh cache for this test to avoid contamination from other tests + PreparedStatementCache testCache = new PreparedStatementCache(); + String insertQuery = "INSERT INTO ks.table (id) VALUES (?)"; + when(session.prepare(insertQuery)).thenReturn(preparedStatement1); + + testCache.getOrPrepare(session, insertQuery); // miss + testCache.getOrPrepare(session, insertQuery); // hit + testCache.getOrPrepare(session, insertQuery); // hit + + CacheStats stats = testCache.getStats(); + // The cache uses getIfPresent first, then get if not present + // So first call: getIfPresent (miss) + get (load) = 2 requests, 1 miss + // Subsequent calls: getIfPresent (hit) = 1 request each, 1 hit each + // Total: 4 requests, 2 hits, 1 miss, 1 load + assertThat(stats.requestCount()).isEqualTo(4); + assertThat(stats.hitCount()).isEqualTo(2); + assertThat(stats.missCount()).isEqualTo(2); + assertThat(stats.loadCount()).isEqualTo(1); + } + + @Test + void testCustomCacheSize() { + PreparedStatementCache customCache = new PreparedStatementCache(500); + String insertQuery = "INSERT INTO ks.table (id) VALUES (?)"; + when(session.prepare(insertQuery)).thenReturn(preparedStatement1); + + PreparedStatement result = customCache.getOrPrepare(session, insertQuery); + assertThat(result).isSameAs(preparedStatement1); + } + + @Test + void testCacheAvoidsBugWithSimilarQueriesDifferentModifiers() { + // This test demonstrates that different queries get different cache entries + String queryWithoutTTL = "INSERT INTO ks.table (id, name) VALUES (?, ?)"; + String queryWithTTL = "INSERT INTO ks.table (id, name) VALUES (?, ?) USING TTL 60"; + + when(session.prepare(queryWithoutTTL)).thenReturn(preparedStatement1); + when(session.prepare(queryWithTTL)).thenReturn(preparedStatement2); + + // Each query should get its own prepared statement using the query as cache key + PreparedStatement result1 = cache.getOrPrepare(session, queryWithoutTTL); + PreparedStatement result2 = cache.getOrPrepare(session, queryWithTTL); + + assertThat(result1).isSameAs(preparedStatement1); + assertThat(result2).isSameAs(preparedStatement2); + assertThat(result1).isNotSameAs(result2); + + // Verify both queries were prepared separately + verify(session, times(1)).prepare(queryWithoutTTL); + verify(session, times(1)).prepare(queryWithTTL); + } + + @Test + void testInputValidation() { + // Null session + assertThatThrownBy(() -> cache.getOrPrepare(null, "query")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("session cannot be null"); + + // Null query + assertThatThrownBy(() -> cache.getOrPrepare(session, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("cqlQuery cannot be empty"); + + // Empty query + assertThatThrownBy(() -> cache.getOrPrepare(session, "")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("cqlQuery cannot be empty"); + } + + @Test + void testConcurrentDifferentQueries() throws InterruptedException { + String query1 = "INSERT INTO ks.table1 (id) VALUES (?)"; + String query2 = "INSERT INTO ks.table2 (name) VALUES (?)"; + when(session.prepare(query1)).thenReturn(preparedStatement1); + when(session.prepare(query2)).thenReturn(preparedStatement2); + + int threadCount = 10; + CountDownLatch startLatch = new CountDownLatch(1); + CountDownLatch finishLatch = new CountDownLatch(threadCount); + ExecutorService executor = Executors.newFixedThreadPool(threadCount); + CompletableFuture[] futures = new CompletableFuture[threadCount]; + + for (int i = 0; i < threadCount; i++) { + final boolean useQuery1 = i % 2 == 0; + futures[i] = + CompletableFuture.supplyAsync( + () -> { + try { + startLatch.await(); + String query = useQuery1 ? query1 : query2; + return cache.getOrPrepare(session, query); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + finishLatch.countDown(); + } + }, + executor); + } + + startLatch.countDown(); + finishLatch.await(); + executor.shutdown(); + + // Verify each query prepared exactly once + verify(session, times(1)).prepare(query1); + verify(session, times(1)).prepare(query2); + + // Verify correct statements returned + for (int i = 0; i < threadCount; i++) { + PreparedStatement expected = i % 2 == 0 ? preparedStatement1 : preparedStatement2; + assertThat(futures[i].join()).isSameAs(expected); + } + } + + @Test + void testCacheEvictionWithSmallSize() { + PreparedStatementCache smallCache = new PreparedStatementCache(2); + when(session.prepare(anyString())) + .thenAnswer( + invocation -> { + // Return a new mock for each unique query + return org.mockito.Mockito.mock(PreparedStatement.class); + }); + + // Fill cache to capacity + smallCache.getOrPrepare(session, "query1"); + smallCache.getOrPrepare(session, "query2"); + + assertThat(smallCache.size()).isLessThanOrEqualTo(2); + + // Add third entry should evict least recently used + smallCache.getOrPrepare(session, "query3"); + assertThat(smallCache.size()).isLessThanOrEqualTo(2); + } + + @Test + void testCloseEmptiesCache() { + String query = "INSERT INTO ks.table (id) VALUES (?)"; + when(session.prepare(query)).thenReturn(preparedStatement1); + + cache.getOrPrepare(session, query); + assertThat(cache.size()).isGreaterThan(0); + + cache.close(); + assertThat(cache.size()).isEqualTo(0); + + // After close, cache should prepare again + cache.getOrPrepare(session, query); + verify(session, times(2)).prepare(query); + } + + @Test + void testErrorNotCached() { + RuntimeException error = new RuntimeException("Prepare failed"); + when(session.prepare(anyString())).thenThrow(error).thenReturn(preparedStatement1); + + // First call fails + assertThatThrownBy(() -> cache.getOrPrepare(session, "query")) + .isInstanceOf(UncheckedExecutionException.class) + .hasCauseInstanceOf(RuntimeException.class) + .hasRootCauseMessage("Prepare failed"); + + // Second call should succeed and prepare again (error not cached) + PreparedStatement result = cache.getOrPrepare(session, "query"); + assertThat(result).isSameAs(preparedStatement1); + verify(session, times(2)).prepare("query"); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NullUnsettingCustomizerTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NullUnsettingCustomizerTest.java new file mode 100644 index 0000000..f8d4332 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/customization/NullUnsettingCustomizerTest.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.customization; + +import com.datastax.driver.core.BoundStatement; +import com.datastax.driver.core.ColumnDefinitions; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Statement; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link NullUnsettingCustomizer}. */ +public class NullUnsettingCustomizerTest { + + @Mock private BoundStatement boundStatement; + @Mock private Statement nonBoundStatement; + @Mock private PreparedStatement preparedStatement; + @Mock private ColumnDefinitions columnDefinitions; + + private NullUnsettingCustomizer customizer; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + customizer = new NullUnsettingCustomizer<>(); + + // Common mock setup for BoundStatement + when(boundStatement.preparedStatement()).thenReturn(preparedStatement); + when(preparedStatement.getVariables()).thenReturn(columnDefinitions); + } + + @Test + void testUnsetsExplicitlySetNulls() { + // Setup: statement has 3 variables, middle one is null + when(columnDefinitions.size()).thenReturn(3); + + when(boundStatement.isSet(0)).thenReturn(true); + when(boundStatement.isSet(1)).thenReturn(true); + when(boundStatement.isSet(2)).thenReturn(true); + + when(boundStatement.isNull(0)).thenReturn(false); + when(boundStatement.isNull(1)).thenReturn(true); + when(boundStatement.isNull(2)).thenReturn(false); + + customizer.apply(boundStatement, "test-record"); + + verify(boundStatement, never()).unset(0); + verify(boundStatement).unset(1); + verify(boundStatement, never()).unset(2); + } + + @Test + void testMultipleNullsUnset() { + // Setup: multiple nulls + when(columnDefinitions.size()).thenReturn(4); + + when(boundStatement.isSet(0)).thenReturn(true); + when(boundStatement.isSet(1)).thenReturn(true); + when(boundStatement.isSet(2)).thenReturn(true); + when(boundStatement.isSet(3)).thenReturn(true); + + when(boundStatement.isNull(0)).thenReturn(true); // null + when(boundStatement.isNull(1)).thenReturn(false); + when(boundStatement.isNull(2)).thenReturn(true); // null + when(boundStatement.isNull(3)).thenReturn(true); // null + + // Execute + customizer.apply(boundStatement, "test-record"); + + // Verify + verify(boundStatement).unset(0); + verify(boundStatement, never()).unset(1); + verify(boundStatement).unset(2); + verify(boundStatement).unset(3); + } + + @Test + void testNonNullValuesRemainSet() { + when(columnDefinitions.size()).thenReturn(3); + + when(boundStatement.isSet(0)).thenReturn(true); + when(boundStatement.isSet(1)).thenReturn(true); + when(boundStatement.isSet(2)).thenReturn(true); + + when(boundStatement.isNull(0)).thenReturn(false); + when(boundStatement.isNull(1)).thenReturn(false); + when(boundStatement.isNull(2)).thenReturn(false); + + // Execute + customizer.apply(boundStatement, "test-record"); + + // Verify nothing was unset + verify(boundStatement, never()).unset(anyInt()); + } + + @Test + void testAlreadyUnsetPositionsRemainUnset() { + when(columnDefinitions.size()).thenReturn(3); + + when(boundStatement.isSet(0)).thenReturn(true); + when(boundStatement.isSet(1)).thenReturn(false); // Already unset + when(boundStatement.isSet(2)).thenReturn(true); + + when(boundStatement.isNull(0)).thenReturn(false); + // isNull(1) should not be called since it's not set + when(boundStatement.isNull(2)).thenReturn(true); + + customizer.apply(boundStatement, "test-record"); + verify(boundStatement, never()).unset(0); + verify(boundStatement, never()).unset(1); // Already unset, not touched + verify(boundStatement).unset(2); + verify(boundStatement, never()).isNull(1); // Should not check null for unset position + } + + @Test + void testNonBoundStatementIsNoOp() { + customizer.apply(nonBoundStatement, "test-record"); + verifyNoInteractions(nonBoundStatement); + } + + @Test + void testZeroVariablesStatement() { + // Setup: statement with no variables + when(columnDefinitions.size()).thenReturn(0); + + // Execute + customizer.apply(boundStatement, "test-record"); + + // Verify no operations performed + verify(boundStatement, never()).isSet(anyInt()); + verify(boundStatement, never()).isNull(anyInt()); + verify(boundStatement, never()).unset(anyInt()); + } + + @Test + void testIdempotency() { + when(columnDefinitions.size()).thenReturn(2); + + when(boundStatement.isSet(0)).thenReturn(true); + when(boundStatement.isSet(1)).thenReturn(true); + when(boundStatement.isNull(0)).thenReturn(false); + when(boundStatement.isNull(1)).thenReturn(true); + + customizer.apply(boundStatement, "test-record"); + + when(boundStatement.isSet(1)).thenReturn(false); + + customizer.apply(boundStatement, "test-record"); + + verify(boundStatement, times(1)).unset(1); + } + + @Test + void testNullStatementReturnsNull() { + customizer.apply(null, "test-record"); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/InsertStrategyTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/InsertStrategyTest.java new file mode 100644 index 0000000..02f4bc2 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/InsertStrategyTest.java @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.strategy; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.types.Row; + +import com.datastax.driver.core.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.QueryBuilder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link InsertStrategy} and {@link StaticInsertStrategy}. + * + *

This test class covers both dynamic INSERT (query builder) and static INSERT (user-provided + * query string) strategies. + */ +class InsertStrategyTest { + + @Mock private CqlSinkConfig config; + @Mock private CqlClauseResolver clauseResolver; + + private InsertStrategy strategy; + private StaticInsertStrategy staticStrategy; + private TableRef table; + private Row record; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + strategy = new InsertStrategy<>(); + staticStrategy = new StaticInsertStrategy<>(); + table = new TableRef("test_keyspace", "test_table"); + record = Row.of(1, "Alice", "alice@example.com"); + } + + // ========== Dynamic InsertStrategy Tests ========== + + @Test + void testSimpleInsertWithoutClauses() { + ResolvedWrite write = + ResolvedWrite.insert( + Arrays.asList("id", "name", "email"), + new Object[] {1, "Alice", "alice@example.com"}); + + when(clauseResolver.applyTo(any(Insert.class), eq(record))) + .thenReturn(ClauseBindings.empty()); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query) + .startsWith("INSERT INTO test_keyspace.test_table") + .contains("(id,name,email) VALUES (?,?,?)"); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly(1, "Alice", "alice@example.com"); + } + + @Test + void testInsertWithTTLClause() { + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + + when(clauseResolver.applyTo(any(Insert.class), eq(record))) + .thenAnswer( + invocation -> { + Insert insert = invocation.getArgument(0); + insert.using(QueryBuilder.ttl(QueryBuilder.bindMarker())); + return new ClauseBindings(new Object[] {3600}, null); + }); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query) + .contains("INSERT INTO test_keyspace.test_table") + .contains("VALUES (?,?)") + .contains("USING TTL ?"); + assertThat(result.clauseBindings.getUsingValues()).containsExactly(3600); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly(1, "Alice", 3600); + } + + @Test + void testInsertWithMultipleClauses() { + ResolvedWrite write = + ResolvedWrite.insert(Collections.singletonList("id"), new Object[] {1}); + + when(clauseResolver.applyTo(any(Insert.class), eq(record))) + .thenAnswer( + invocation -> { + Insert insert = invocation.getArgument(0); + insert.using(QueryBuilder.ttl(QueryBuilder.bindMarker())) + .and(QueryBuilder.timestamp(QueryBuilder.bindMarker())); + return new ClauseBindings(new Object[] {7200, 1234567890L}, null); + }); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).contains("USING TTL ?").contains("AND TIMESTAMP ?"); + assertThat(result.clauseBindings.getUsingValues()).containsExactly(7200, 1234567890L); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly(1, 7200, 1234567890L); + } + + @Test + void testInsertWithIfNotExists() { + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + + when(clauseResolver.applyTo(any(Insert.class), eq(record))) + .thenAnswer( + invocation -> { + Insert insert = invocation.getArgument(0); + insert.ifNotExists(); + return ClauseBindings.empty(); + }); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).contains("IF NOT EXISTS"); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly(1, "Alice"); + } + + @Test + void testInsertFailsForUpdateOperation() { + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("name"), + new Object[] {"Bob"}, + Collections.singletonList("id"), + new Object[] {1}); + + assertThatThrownBy( + () -> + strategy.getQueryWithBindings( + table, config, clauseResolver, write, record)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("InsertStrategy received UPDATE"); + } + + @Test + void testGetOrderedBindValuesWithEmptyClauseBindings() { + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + + Object[] bindValues = strategy.getOrderedBindValues(write, ClauseBindings.empty()); + assertThat(bindValues).containsExactly(1, "Alice"); + } + + @Test + void testGetOrderedBindValuesWithClauseBindings() { + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + + ClauseBindings clauseBindings = new ClauseBindings(new Object[] {3600, 1234567890L}, null); + Object[] bindValues = strategy.getOrderedBindValues(write, clauseBindings); + assertThat(bindValues).containsExactly(1, "Alice", 3600, 1234567890L); + } + + // ========== Static InsertStrategy Tests ========== + + @Test + void testStaticInsertQuery() { + String staticQuery = + "INSERT INTO test_keyspace.test_table (id, name, email) VALUES (?, ?, ?)"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.insert( + Arrays.asList("id", "name", "email"), + new Object[] {1, "Alice", "alice@example.com"}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly(1, "Alice", "alice@example.com"); + } + + @Test + void testStaticInsertWithTTL() { + String staticQuery = + "INSERT INTO test_keyspace.test_table (id, name) VALUES (?, ?) USING TTL 3600"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly(1, "Alice"); + } + + @Test + void testStaticInsertWithIfNotExists() { + String staticQuery = + "INSERT INTO test_keyspace.test_table (id, name) VALUES (?, ?) IF NOT EXISTS"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly(1, "Alice"); + } + + @Test + void testStaticInsertFailsForUpdateOperation() { + String staticQuery = "INSERT INTO test_keyspace.test_table (id, name) VALUES (?, ?)"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("name"), + new Object[] {"Bob"}, + Collections.singletonList("id"), + new Object[] {1}); + + assertThatThrownBy( + () -> + staticStrategy.getQueryWithBindings( + table, config, clauseResolver, write, record)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("StaticInsertStrategy got UPDATE"); + } + + @Test + void testStaticInsertWithComplexQuery() { + String staticQuery = + "INSERT INTO test_keyspace.test_table (id, data) VALUES (?, ?) " + + "USING TTL 7200 AND TIMESTAMP 1234567890"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "data"), new Object[] {1, "complex_data"}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly(1, "complex_data"); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/UpdateStrategyTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/UpdateStrategyTest.java new file mode 100644 index 0000000..6a7d875 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/core/strategy/UpdateStrategyTest.java @@ -0,0 +1,455 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.core.strategy; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.ClauseBindings; +import org.apache.flink.connector.cassandra.sink.planner.core.modifiers.CqlClauseResolver; +import org.apache.flink.connector.cassandra.sink.planner.resolver.ResolvedWrite; +import org.apache.flink.connector.cassandra.sink.planner.resolver.TableRef; +import org.apache.flink.types.Row; + +import com.datastax.driver.core.querybuilder.QueryBuilder; +import com.datastax.driver.core.querybuilder.Update; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link UpdateStrategy} and {@link StaticUpdateStrategy}. + * + *

This test class covers both dynamic UPDATE (query builder) and static UPDATE (user-provided + * query string) strategies. + */ +class UpdateStrategyTest { + + @Mock private CqlSinkConfig config; + @Mock private CqlClauseResolver clauseResolver; + + private UpdateStrategy strategy; + private StaticUpdateStrategy staticStrategy; + private TableRef table; + private Row record; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + strategy = new UpdateStrategy<>(); + staticStrategy = new StaticUpdateStrategy<>(); + table = new TableRef("test_keyspace", "test_table"); + record = Row.of("Bob", 25, 1); + } + + // ========== Dynamic UpdateStrategy Tests ========== + + @Test + void testSimpleUpdateWithoutClauses() { + ResolvedWrite write = + ResolvedWrite.update( + Arrays.asList("name", "age"), + new Object[] {"Bob", 25}, + Collections.singletonList("id"), + new Object[] {1}); + + when(clauseResolver.applyTo(any(Update.class), eq(record))) + .thenReturn(ClauseBindings.empty()); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query) + .isEqualTo("UPDATE test_keyspace.test_table SET name=?,age=? WHERE id=?;"); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly("Bob", 25, 1); + } + + @Test + void testUpdateWithUsingTTL() { + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("name"), + new Object[] {"Bob"}, + Collections.singletonList("id"), + new Object[] {1}); + + when(clauseResolver.applyTo(any(Update.class), eq(record))) + .thenAnswer( + invocation -> { + Update update = invocation.getArgument(0); + update.using(QueryBuilder.ttl(QueryBuilder.bindMarker())); + return new ClauseBindings(new Object[] {3600}, null); + }); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query) + .contains("UPDATE test_keyspace.test_table") + .contains("USING TTL ?") + .contains("SET name=?") + .contains("WHERE id=?"); + assertThat(result.clauseBindings.getUsingValues()).containsExactly(3600); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + // For UPDATE with USING, the order is: USING values, SET values, WHERE values + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly(3600, "Bob", 1); + } + + @Test + void testUpdateWithUsingTimestamp() { + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("status"), + new Object[] {"active"}, + Collections.singletonList("id"), + new Object[] {123}); + + when(clauseResolver.applyTo(any(Update.class), eq(record))) + .thenAnswer( + invocation -> { + Update update = invocation.getArgument(0); + update.using(QueryBuilder.timestamp(QueryBuilder.bindMarker())); + return new ClauseBindings(new Object[] {1234567890L}, null); + }); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).contains("USING TIMESTAMP ?"); + assertThat(result.clauseBindings.getUsingValues()).containsExactly(1234567890L); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly(1234567890L, "active", 123); + } + + @Test + void testUpdateWithIfCondition() { + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("status"), + new Object[] {"completed"}, + Collections.singletonList("id"), + new Object[] {456}); + + when(clauseResolver.applyTo(any(Update.class), eq(record))) + .thenAnswer( + invocation -> { + Update update = invocation.getArgument(0); + update.onlyIf(QueryBuilder.eq("status", QueryBuilder.bindMarker())); + return new ClauseBindings(null, new Object[] {"pending"}); + }); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query) + .contains("UPDATE test_keyspace.test_table") + .contains("SET status=?") + .contains("WHERE id=?") + .contains("IF status=?"); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).containsExactly("pending"); + + // For UPDATE with IF, the order is: SET values, WHERE values, IF values + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly("completed", 456, "pending"); + } + + @Test + void testUpdateWithMultipleWhereColumns() { + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("status"), + new Object[] {"active"}, + Arrays.asList("partition_key", "clustering_key"), + new Object[] {"partition1", "cluster1"}); + + when(clauseResolver.applyTo(any(Update.class), eq(record))) + .thenReturn(ClauseBindings.empty()); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query) + .isEqualTo( + "UPDATE test_keyspace.test_table SET status=? WHERE partition_key=? AND clustering_key=?;"); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly("active", "partition1", "cluster1"); + } + + @Test + void testUpdateWithNullValues() { + ResolvedWrite write = + ResolvedWrite.update( + Arrays.asList("name", "email"), + new Object[] {"Bob", null}, + Collections.singletonList("id"), + new Object[] {1}); + + when(clauseResolver.applyTo(any(Update.class), eq(record))) + .thenReturn(ClauseBindings.empty()); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + assertThat(bindValues).containsExactly("Bob", null, 1); + } + + @Test + void testUpdateFailsForInsertOperation() { + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + + assertThatThrownBy( + () -> + strategy.getQueryWithBindings( + table, config, clauseResolver, write, record)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("UpdateStrategy received INSERT"); + } + + @Test + void testGetOrderedBindValuesWithUsingClause() { + ResolvedWrite write = + ResolvedWrite.update( + Arrays.asList("name", "age"), + new Object[] {"Bob", 25}, + Collections.singletonList("id"), + new Object[] {1}); + + // Simulate USING TTL and TIMESTAMP + ClauseBindings clauseBindings = new ClauseBindings(new Object[] {3600, 1234567890L}, null); + Object[] bindValues = strategy.getOrderedBindValues(write, clauseBindings); + + // USING values come first for UPDATE + assertThat(bindValues).containsExactly(3600, 1234567890L, "Bob", 25, 1); + } + + @Test + void testGetOrderedBindValuesWithIfClause() { + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("status"), + new Object[] {"completed"}, + Collections.singletonList("id"), + new Object[] {456}); + + // IF clause values + ClauseBindings clauseBindings = new ClauseBindings(null, new Object[] {"pending"}); + Object[] bindValues = strategy.getOrderedBindValues(write, clauseBindings); + + // IF values come last for UPDATE + assertThat(bindValues).containsExactly("completed", 456, "pending"); + } + + @Test + void testComplexUpdateWithMultipleClauses() { + ResolvedWrite write = + ResolvedWrite.update( + Arrays.asList("name", "age", "status"), + new Object[] {"Bob", 25, "active"}, + Arrays.asList("partition_key", "clustering_key"), + new Object[] {"pk1", "ck1"}); + + when(clauseResolver.applyTo(any(Update.class), eq(record))) + .thenAnswer( + invocation -> { + Update update = invocation.getArgument(0); + update.using(QueryBuilder.ttl(QueryBuilder.bindMarker())) + .and(QueryBuilder.timestamp(QueryBuilder.bindMarker())) + .onlyIf(QueryBuilder.eq("version", QueryBuilder.bindMarker())); + return new ClauseBindings( + new Object[] {7200, 9876543210L}, new Object[] {5}); + }); + + PlannerStrategy.QueryWithBindings result = + strategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query) + .contains("USING TTL ?") + .contains("AND TIMESTAMP ?") + .contains("IF version=?"); + assertThat(result.clauseBindings.getUsingValues()).containsExactly(7200, 9876543210L); + assertThat(result.clauseBindings.getIfValues()).containsExactly(5); + + Object[] bindValues = strategy.getOrderedBindValues(write, result.clauseBindings); + // USING values first, then SET, then WHERE, then IF + assertThat(bindValues) + .containsExactly(7200, 9876543210L, "Bob", 25, "active", "pk1", "ck1", 5); + } + + // ========== Static UpdateStrategy Tests ========== + + @Test + void testStaticUpdateQuery() { + String staticQuery = "UPDATE test_keyspace.test_table SET name=?, age=? WHERE id=?"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.update( + Arrays.asList("name", "age"), + new Object[] {"Bob", 25}, + Collections.singletonList("id"), + new Object[] {1}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly("Bob", 25, 1); + } + + @Test + void testStaticUpdateWithTTL() { + String staticQuery = + "UPDATE test_keyspace.test_table USING TTL 3600 SET status=? WHERE id=?"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("status"), + new Object[] {"active"}, + Collections.singletonList("id"), + new Object[] {1}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly("active", 1); + } + + @Test + void testStaticUpdateWithIfCondition() { + String staticQuery = "UPDATE test_keyspace.test_table SET status=? WHERE id=? IF status=?"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("status"), + new Object[] {"completed"}, + Collections.singletonList("id"), + new Object[] {456}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly("completed", 456); + } + + @Test + void testStaticUpdateFailsForInsertOperation() { + String staticQuery = "UPDATE test_keyspace.test_table SET name=? WHERE id=?"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {1, "Alice"}); + + assertThatThrownBy( + () -> + staticStrategy.getQueryWithBindings( + table, config, clauseResolver, write, record)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("StaticUpdateStrategy got INSERT"); + } + + @Test + void testStaticUpdateWithMultipleWhereColumns() { + String staticQuery = + "UPDATE test_keyspace.test_table SET status=? WHERE partition_key=? AND clustering_key=?"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("status"), + new Object[] {"active"}, + Arrays.asList("partition_key", "clustering_key"), + new Object[] {"partition1", "cluster1"}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly("active", "partition1", "cluster1"); + } + + @Test + void testStaticUpdateWithComplexQuery() { + String staticQuery = + "UPDATE test_keyspace.test_table USING TTL 7200 AND TIMESTAMP 1234567890 " + + "SET data=? WHERE id=? IF EXISTS"; + when(config.getQuery()).thenReturn(staticQuery); + + ResolvedWrite write = + ResolvedWrite.update( + Collections.singletonList("data"), + new Object[] {"complex_data"}, + Collections.singletonList("id"), + new Object[] {1}); + + PlannerStrategy.QueryWithBindings result = + staticStrategy.getQueryWithBindings(table, config, clauseResolver, write, record); + + assertThat(result.query).isEqualTo(staticQuery); + assertThat(result.clauseBindings).isNotNull(); + assertThat(result.clauseBindings.getUsingValues()).isEmpty(); + assertThat(result.clauseBindings.getIfValues()).isEmpty(); + + Object[] bindValues = staticStrategy.getOrderedBindValues(write, null); + assertThat(bindValues).containsExactly("complex_data", 1); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ColumnValueResolverTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ColumnValueResolverTest.java new file mode 100644 index 0000000..629b19b --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ColumnValueResolverTest.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for {@link ColumnValueResolver} interface and its contract. */ +class ColumnValueResolverTest { + + @Test + void testInsertResolverContract() { + ColumnValueResolver insertResolver = new TestInsertResolver(); + + assertThat(insertResolver.kind()).isEqualTo(ColumnValueResolver.Kind.INSERT); + + Row record = Row.of(123, "Alice", "alice@example.com"); + ResolvedWrite result = insertResolver.resolve(record); + + assertThat(result.isUpdate()).isFalse(); + assertThat(result.setColumns()).containsExactly("id", "name", "email"); + assertThat(result.setValues()).containsExactly(123, "Alice", "alice@example.com"); + assertThat(result.whereColumns()).isEmpty(); + assertThat(result.whereValues()).isEmpty(); + } + + @Test + void testUpdateResolverContract() { + ColumnValueResolver updateResolver = new TestUpdateResolver(); + + assertThat(updateResolver.kind()).isEqualTo(ColumnValueResolver.Kind.UPDATE); + + Row record = Row.of(123, "Alice Updated", "alice.updated@example.com"); + ResolvedWrite result = updateResolver.resolve(record); + + assertThat(result.isUpdate()).isTrue(); + assertThat(result.setColumns()).containsExactly("name", "email"); + assertThat(result.setValues()) + .containsExactly("Alice Updated", "alice.updated@example.com"); + assertThat(result.whereColumns()).containsExactly("id"); + assertThat(result.whereValues()).containsExactly(123); + } + + @Test + void testConditionalResolver() { + ColumnValueResolver conditionalResolver = new TestConditionalResolver(); + + assertThat(conditionalResolver.kind()) + .isEqualTo(ColumnValueResolver.Kind.INSERT); // Default/primary + + // INSERT case: update_flag is null + Row insertRecord = Row.of(123, "Alice", "alice@example.com", null); + ResolvedWrite insertResult = conditionalResolver.resolve(insertRecord); + + assertThat(insertResult.isUpdate()).isFalse(); + assertThat(insertResult.setColumns()).containsExactly("id", "name", "email"); + assertThat(insertResult.setValues()).containsExactly(123, "Alice", "alice@example.com"); + + // UPDATE case: update_flag is not null + Row updateRecord = Row.of(123, "Alice Updated", "alice.updated@example.com", "UPDATE"); + ResolvedWrite updateResult = conditionalResolver.resolve(updateRecord); + + assertThat(updateResult.isUpdate()).isTrue(); + assertThat(updateResult.setColumns()).containsExactly("name", "email"); + assertThat(updateResult.setValues()) + .containsExactly("Alice Updated", "alice.updated@example.com"); + assertThat(updateResult.whereColumns()).containsExactly("id"); + assertThat(updateResult.whereValues()).containsExactly(123); + } + + // Test implementations demonstrating the interface contract + + private static class TestInsertResolver implements ColumnValueResolver { + + @Override + public Kind kind() { + return Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + return ResolvedWrite.insert( + Arrays.asList("id", "name", "email"), + new Object[] { + record.getField(0), // id + record.getField(1), // name + record.getField(2) // email + }); + } + } + + private static class TestUpdateResolver implements ColumnValueResolver { + + @Override + public Kind kind() { + return Kind.UPDATE; + } + + @Override + public ResolvedWrite resolve(Row record) { + return ResolvedWrite.update( + Arrays.asList("name", "email"), // SET columns + new Object[] { + record.getField(1), // name + record.getField(2) // email + }, + Collections.singletonList("id"), // WHERE columns + new Object[] { + record.getField(0) // id + }); + } + } + + private static class TestConditionalResolver implements ColumnValueResolver { + + @Override + public Kind kind() { + // Return primary/default kind - this resolver can handle both + return Kind.INSERT; + } + + @Override + public ResolvedWrite resolve(Row record) { + boolean isUpdate = record.getField(3) != null; // has update flag + + if (isUpdate) { + return ResolvedWrite.update( + Arrays.asList("name", "email"), + new Object[] {record.getField(1), record.getField(2)}, + Collections.singletonList("id"), + new Object[] {record.getField(0)}); + } else { + return ResolvedWrite.insert( + Arrays.asList("id", "name", "email"), + new Object[] {record.getField(0), record.getField(1), record.getField(2)}); + } + } + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedColumnValueResolverTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedColumnValueResolverTest.java new file mode 100644 index 0000000..27807f2 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/FixedColumnValueResolverTest.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.connector.cassandra.sink.config.RecordFormatType; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Unit tests for {@link FixedColumnValueResolver}. */ +class FixedColumnValueResolverTest { + + @Test + void testInsertWithRow() { + FixedColumnValueResolver resolver = + new FixedColumnValueResolver<>( + RecordFormatType.ROW, Arrays.asList("id", "name", "email")); + + assertThat(resolver.kind()).isEqualTo(ColumnValueResolver.Kind.INSERT); + + Row record = Row.of(123, "Alice", "alice@example.com"); + ResolvedWrite result = resolver.resolve(record); + + assertThat(result.isUpdate()).isFalse(); + assertThat(result.setColumns()).containsExactly("id", "name", "email"); + assertThat(result.setValues()).containsExactly(123, "Alice", "alice@example.com"); + assertThat(result.whereColumns()).isEmpty(); + assertThat(result.whereValues()).isEmpty(); + } + + @Test + void testInsertWithTuple() { + FixedColumnValueResolver> resolver = + new FixedColumnValueResolver<>( + RecordFormatType.TUPLE, Arrays.asList("id", "name", "email")); + + assertThat(resolver.kind()).isEqualTo(ColumnValueResolver.Kind.INSERT); + + Tuple3 record = new Tuple3<>(123, "Bob", "bob@example.com"); + ResolvedWrite result = resolver.resolve(record); + + assertThat(result.isUpdate()).isFalse(); + assertThat(result.setColumns()).containsExactly("id", "name", "email"); + assertThat(result.setValues()).containsExactly(123, "Bob", "bob@example.com"); + assertThat(result.whereColumns()).isEmpty(); + assertThat(result.whereValues()).isEmpty(); + } + + @Test + void testInsertSingleColumn() { + FixedColumnValueResolver resolver = + new FixedColumnValueResolver<>( + RecordFormatType.ROW, Collections.singletonList("id")); + + Row record = Row.of(456); + ResolvedWrite result = resolver.resolve(record); + + assertThat(result.setColumns()).containsExactly("id"); + assertThat(result.setValues()).containsExactly(456); + } + + @Test + void testInsertFieldCountMismatch() { + FixedColumnValueResolver resolver = + new FixedColumnValueResolver<>(RecordFormatType.ROW, Arrays.asList("id", "name")); + + Row record = Row.of(123, "Alice", "extra_field"); + + assertThatThrownBy(() -> resolver.resolve(record)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("setColumns size (2) must equal setValues length (3)"); + } + + @Test + void testUpdateWithRow() { + FixedColumnValueResolver resolver = + new FixedColumnValueResolver<>( + RecordFormatType.ROW, + Arrays.asList("name", "email"), // SET columns + Arrays.asList("id", "version")); // WHERE columns + + assertThat(resolver.kind()).isEqualTo(ColumnValueResolver.Kind.UPDATE); + + Row record = Row.of("Alice Updated", "alice.updated@example.com", 123, 1); + ResolvedWrite result = resolver.resolve(record); + + assertThat(result.isUpdate()).isTrue(); + assertThat(result.setColumns()).containsExactly("name", "email"); + assertThat(result.setValues()) + .containsExactly("Alice Updated", "alice.updated@example.com"); + assertThat(result.whereColumns()).containsExactly("id", "version"); + assertThat(result.whereValues()).containsExactly(123, 1); + } + + @Test + void testUpdateWithTuple() { + // Tuple4 for: name, email (SET), id, version (WHERE) + FixedColumnValueResolver> resolver = + new FixedColumnValueResolver<>( + RecordFormatType.TUPLE, + Collections.singletonList("name"), // SET columns + Arrays.asList("id", "version")); // WHERE columns + + Tuple3 record = new Tuple3<>("Bob Updated", 123, 2); + ResolvedWrite result = resolver.resolve(record); + + assertThat(result.isUpdate()).isTrue(); + assertThat(result.setColumns()).containsExactly("name"); + assertThat(result.setValues()).containsExactly("Bob Updated"); + assertThat(result.whereColumns()).containsExactly("id", "version"); + assertThat(result.whereValues()).containsExactly(123, 2); + } + + @Test + void testUpdateSingleColumns() { + FixedColumnValueResolver resolver = + new FixedColumnValueResolver<>( + RecordFormatType.ROW, + Collections.singletonList("status"), // SET + Collections.singletonList("id")); // WHERE + + Row record = Row.of("active", 789); + ResolvedWrite result = resolver.resolve(record); + + assertThat(result.setColumns()).containsExactly("status"); + assertThat(result.setValues()).containsExactly("active"); + assertThat(result.whereColumns()).containsExactly("id"); + assertThat(result.whereValues()).containsExactly(789); + } + + @Test + void testUpdateFieldCountMismatch() { + FixedColumnValueResolver resolver = + new FixedColumnValueResolver<>( + RecordFormatType.ROW, + Arrays.asList("name", "email"), // 2 SET columns + Collections.singletonList("id")); // 1 WHERE column (total: 3) + + Row record = Row.of("Alice", "alice@example.com"); // Only 2 fields + + assertThatThrownBy(() -> resolver.resolve(record)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "UPDATE parameter count mismatch: query expects 3 parameters (2 SET + 1 WHERE) but record has 2 fields"); + } + + @Test + void testNullValues() { + FixedColumnValueResolver resolver = + new FixedColumnValueResolver<>( + RecordFormatType.ROW, Arrays.asList("id", "optional_field")); + + Row record = Row.of(123, null); + ResolvedWrite result = resolver.resolve(record); + + assertThat(result.setValues()).containsExactly(123, null); + } + + @Test + void testDifferentRecordFormatTypes() { + // Test that different RecordFormatType values are accepted + FixedColumnValueResolver rowResolver = + new FixedColumnValueResolver<>( + RecordFormatType.ROW, Collections.singletonList("id")); + + FixedColumnValueResolver> tupleResolver = + new FixedColumnValueResolver<>( + RecordFormatType.TUPLE, Collections.singletonList("id")); + + assertThat(rowResolver.kind()).isEqualTo(ColumnValueResolver.Kind.INSERT); + assertThat(tupleResolver.kind()).isEqualTo(ColumnValueResolver.Kind.INSERT); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ResolvedWriteTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ResolvedWriteTest.java new file mode 100644 index 0000000..84373a7 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/ResolvedWriteTest.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Unit tests for {@link ResolvedWrite}. */ +class ResolvedWriteTest { + + @Test + void testInsertSingleColumn() { + ResolvedWrite insert = + ResolvedWrite.insert(Collections.singletonList("id"), new Object[] {123}); + + assertThat(insert.isUpdate()).isFalse(); + assertThat(insert.setColumns()).containsExactly("id"); + assertThat(insert.setValues()).containsExactly(123); + assertThat(insert.whereColumns()).isEmpty(); + assertThat(insert.whereValues()).isEmpty(); + } + + @Test + void testInsertMultipleColumns() { + ResolvedWrite insert = + ResolvedWrite.insert( + Arrays.asList("id", "name", "email"), + new Object[] {101, "Alice", "alice@example.com"}); + + assertThat(insert.isUpdate()).isFalse(); + assertThat(insert.setColumns()).containsExactly("id", "name", "email"); + assertThat(insert.setValues()).containsExactly(101, "Alice", "alice@example.com"); + assertThat(insert.whereColumns()).isEmpty(); + assertThat(insert.whereValues()).isEmpty(); + } + + @Test + void testInsertNullColumns() { + assertThatThrownBy(() -> ResolvedWrite.insert(null, new Object[] {123})) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("setColumns cannot be null"); + } + + @Test + void testInsertNullValues() { + assertThatThrownBy(() -> ResolvedWrite.insert(Collections.singletonList("id"), null)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("setValues cannot be null"); + } + + @Test + void testInsertEmptyColumns() { + assertThatThrownBy(() -> ResolvedWrite.insert(Collections.emptyList(), new Object[] {})) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("setColumns cannot be empty"); + } + + @Test + void testInsertMismatchedCounts() { + assertThatThrownBy( + () -> ResolvedWrite.insert(Arrays.asList("id", "name"), new Object[] {123})) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("setColumns size (2) must equal setValues length (1)"); + } + + @Test + void testUpdateSingleColumns() { + ResolvedWrite update = + ResolvedWrite.update( + Collections.singletonList("name"), + new Object[] {"Bob"}, + Collections.singletonList("id"), + new Object[] {123}); + + assertThat(update.isUpdate()).isTrue(); + assertThat(update.setColumns()).containsExactly("name"); + assertThat(update.setValues()).containsExactly("Bob"); + assertThat(update.whereColumns()).containsExactly("id"); + assertThat(update.whereValues()).containsExactly(123); + } + + @Test + void testUpdateMultipleColumns() { + ResolvedWrite update = + ResolvedWrite.update( + Arrays.asList("name", "email"), + new Object[] {"Bob Updated", "bob@example.com"}, + Arrays.asList("id", "version"), + new Object[] {123, 1}); + + assertThat(update.isUpdate()).isTrue(); + assertThat(update.setColumns()).containsExactly("name", "email"); + assertThat(update.setValues()).containsExactly("Bob Updated", "bob@example.com"); + assertThat(update.whereColumns()).containsExactly("id", "version"); + assertThat(update.whereValues()).containsExactly(123, 1); + } + + @Test + void testUpdateOverlappingColumns() { + assertThatThrownBy( + () -> + ResolvedWrite.update( + Arrays.asList("id", "name"), + new Object[] {123, "Alice"}, + Arrays.asList("id", "version"), + new Object[] {123, 1})) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Column 'id' cannot appear in both SET and WHERE clauses"); + } + + @Test + void testUpdateMismatchedSetCounts() { + assertThatThrownBy( + () -> + ResolvedWrite.update( + Arrays.asList("name", "email"), + new Object[] {"Bob"}, + Collections.singletonList("id"), + new Object[] {123})) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("setColumns size (2) must equal setValues length (1)"); + } + + @Test + void testUpdateMismatchedWhereCounts() { + assertThatThrownBy( + () -> + ResolvedWrite.update( + Collections.singletonList("name"), + new Object[] {"Bob"}, + Arrays.asList("id", "version"), + new Object[] {123})) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("whereColumns size (2) must equal whereValues length (1)"); + } + + @Test + void testUpdateEmptySetColumns() { + assertThatThrownBy( + () -> + ResolvedWrite.update( + Collections.emptyList(), + new Object[] {}, + Collections.singletonList("id"), + new Object[] {123})) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("setColumns cannot be empty"); + } + + @Test + void testUpdateWithEmptyWhere() { + ResolvedWrite update = + ResolvedWrite.update( + Collections.singletonList("status"), + new Object[] {"active"}, + Collections.emptyList(), + new Object[] {}); + + assertThat(update.isUpdate()).isFalse(); // Empty WHERE means it's treated like INSERT + assertThat(update.setColumns()).containsExactly("status"); + assertThat(update.setValues()).containsExactly("active"); + assertThat(update.whereColumns()).isEmpty(); + assertThat(update.whereValues()).isEmpty(); + } + + @Test + void testNullValuesInArrays() { + ResolvedWrite insert = + ResolvedWrite.insert( + Arrays.asList("id", "optional_field"), new Object[] {123, null}); + + assertThat(insert.setValues()).containsExactly(123, null); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableRefTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableRefTest.java new file mode 100644 index 0000000..cf583a1 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/planner/resolver/TableRefTest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.planner.resolver; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Unit tests for {@link TableRef}. */ +class TableRefTest { + + @Test + void testValidConstruction() { + TableRef tableRef = new TableRef("my_keyspace", "my_table"); + + assertThat(tableRef.keyspace()).isEqualTo("my_keyspace"); + assertThat(tableRef.tableName()).isEqualTo("my_table"); + assertThat(tableRef.getFullyQualifiedName()).isEqualTo("my_keyspace.my_table"); + assertThat(tableRef.toString()).isEqualTo("my_keyspace.my_table"); + } + + @Test + void testInvalidParametersFail() { + // Null keyspace + assertThatThrownBy(() -> new TableRef(null, "my_table")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("keyspace cannot be null/empty"); + + // Empty keyspace + assertThatThrownBy(() -> new TableRef("", "my_table")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("keyspace cannot be null/empty"); + + // Whitespace keyspace + assertThatThrownBy(() -> new TableRef(" ", "my_table")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("keyspace cannot be null/empty"); + + // Null table name + assertThatThrownBy(() -> new TableRef("my_keyspace", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("tableName cannot be null/empty"); + + // Empty table name + assertThatThrownBy(() -> new TableRef("my_keyspace", "")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("tableName cannot be null/empty"); + + // Whitespace table name + assertThatThrownBy(() -> new TableRef("my_keyspace", " ")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("tableName cannot be null/empty"); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/CqlStatementHelperTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/CqlStatementHelperTest.java new file mode 100644 index 0000000..540fc53 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/CqlStatementHelperTest.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.util; + +import org.apache.flink.api.java.tuple.Tuple0; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.connector.cassandra.sink.config.RecordFormatType; +import org.apache.flink.types.Row; + +import com.datastax.driver.core.BoundStatement; +import com.datastax.driver.core.PreparedStatement; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link CqlStatementHelper}. */ +public class CqlStatementHelperTest { + + @Mock private PreparedStatement preparedStatement; + @Mock private BoundStatement boundStatement; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + } + + @Test + void testExtractFieldsTupleVariousArities() { + // Tuple3 with values returns array in index order + Tuple3 tuple3 = new Tuple3<>(1, "a", null); + Object[] result3 = CqlStatementHelper.extractFields(tuple3, RecordFormatType.TUPLE); + assertThat(result3).containsExactly(1, "a", null); + + // Tuple0 returns empty array + Tuple0 tuple0 = new Tuple0(); + Object[] result0 = CqlStatementHelper.extractFields(tuple0, RecordFormatType.TUPLE); + assertThat(result0).isEmpty(); + + // Tuple1 with null preserves null + Tuple1 tuple1 = new Tuple1<>(null); + Object[] result1 = CqlStatementHelper.extractFields(tuple1, RecordFormatType.TUPLE); + assertThat(result1).containsExactly((Object) null); + } + + @Test + void testExtractFieldsRowVariousCases() { + // Wrong pairing: Row with TUPLE format throws ClassCastException + Row wrongRow = Row.of(1, 2, 3); + assertThatThrownBy(() -> CqlStatementHelper.extractFields(wrongRow, RecordFormatType.TUPLE)) + .isInstanceOf(ClassCastException.class); + + // Row with arity N returns values in order + Row row = Row.of(1, "test", true, 42L); + Object[] result = CqlStatementHelper.extractFields(row, RecordFormatType.ROW); + assertThat(result).containsExactly(1, "test", true, 42L); + + // Row with internal nulls preserves nulls + Row rowWithNulls = Row.of(1, null, "test", null); + Object[] resultWithNulls = + CqlStatementHelper.extractFields(rowWithNulls, RecordFormatType.ROW); + assertThat(resultWithNulls).containsExactly(1, null, "test", null); + + // Row with mixed types preserves types and order + Row mixedRow = Row.of(42, "string", true, 3.14, 999L, (byte) 7); + Object[] mixedResult = CqlStatementHelper.extractFields(mixedRow, RecordFormatType.ROW); + assertThat(mixedResult).containsExactly(42, "string", true, 3.14, 999L, (byte) 7); + // Verify exact types preserved + assertThat(mixedResult[0]).isInstanceOf(Integer.class); + assertThat(mixedResult[1]).isInstanceOf(String.class); + assertThat(mixedResult[2]).isInstanceOf(Boolean.class); + assertThat(mixedResult[3]).isInstanceOf(Double.class); + assertThat(mixedResult[4]).isInstanceOf(Long.class); + assertThat(mixedResult[5]).isInstanceOf(Byte.class); + + // Empty Row returns empty array + Row emptyRow = Row.of(); + Object[] emptyResult = CqlStatementHelper.extractFields(emptyRow, RecordFormatType.ROW); + assertThat(emptyResult).isEmpty(); + } + + @Test + void testExtractFieldsScalaProductVariousCases() { + // ScalaProduct3 returns values in order + TestScalaProduct product3 = new TestScalaProduct("v0", 42, true); + Object[] result3 = + CqlStatementHelper.extractFields(product3, RecordFormatType.SCALA_PRODUCT); + assertThat(result3).containsExactly("v0", 42, true); + + // ScalaProduct0 returns empty array + TestScalaProduct0 product0 = new TestScalaProduct0(); + Object[] result0 = + CqlStatementHelper.extractFields(product0, RecordFormatType.SCALA_PRODUCT); + assertThat(result0).isEmpty(); + + // Wrong pairing: Tuple with SCALA_PRODUCT format throws ClassCastException + Tuple3 tuple = new Tuple3<>("a", 1, true); + assertThatThrownBy( + () -> + CqlStatementHelper.extractFields( + tuple, RecordFormatType.SCALA_PRODUCT)) + .isInstanceOf(ClassCastException.class); + } + + @Test + void testExtractFieldsPojoFormatThrowsIllegalArgumentException() { + Object record = new Object(); + + assertThatThrownBy(() -> CqlStatementHelper.extractFields(record, RecordFormatType.POJO)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unsupported RecordFormatType: POJO"); + } + + @Test + void testExtractFieldsLargeArityRowReturnsAllValuesInOrder() { + // Create a row with 100 fields + Object[] values = new Object[100]; + for (int i = 0; i < 100; i++) { + values[i] = "field" + i; + } + Row largeRow = Row.of(values); + + Object[] result = CqlStatementHelper.extractFields(largeRow, RecordFormatType.ROW); + + assertThat(result).hasSize(100); + for (int i = 0; i < 100; i++) { + assertThat(result[i]).isEqualTo("field" + i); + } + } + + @Test + void testExtractFieldsPreservesSameReferences() { + String sharedString = "shared"; + Integer sharedInt = 42; + Row row = Row.of(sharedString, sharedInt); + + Object[] result = CqlStatementHelper.extractFields(row, RecordFormatType.ROW); + assertThat(result[0]).isSameAs(sharedString); + assertThat(result[1]).isSameAs(sharedInt); + } + + @Test + void testBindVariousArrays() { + // Binds positional array and returns BoundStatement + Object[] values = {"a", "b", "c"}; + when(preparedStatement.bind(values)).thenReturn(boundStatement); + BoundStatement result = CqlStatementHelper.bind(preparedStatement, values); + assertThat(result).isSameAs(boundStatement); + verify(preparedStatement).bind(values); + + // Binds empty array and returns BoundStatement + Object[] emptyValues = new Object[0]; + when(preparedStatement.bind(emptyValues)).thenReturn(boundStatement); + BoundStatement emptyResult = CqlStatementHelper.bind(preparedStatement, emptyValues); + assertThat(emptyResult).isNotNull(); + assertThat(emptyResult).isSameAs(boundStatement); + verify(preparedStatement).bind(emptyValues); + + // Binds array with null elements + Object[] nullValues = {"a", null, "c", null}; + when(preparedStatement.bind(nullValues)).thenReturn(boundStatement); + BoundStatement nullResult = CqlStatementHelper.bind(preparedStatement, nullValues); + assertThat(nullResult).isSameAs(boundStatement); + verify(preparedStatement).bind(nullValues); + + // Binds large array + Object[] largeValues = new Object[100]; + for (int i = 0; i < 100; i++) { + largeValues[i] = i; + } + when(preparedStatement.bind(largeValues)).thenReturn(boundStatement); + BoundStatement largeResult = CqlStatementHelper.bind(preparedStatement, largeValues); + assertThat(largeResult).isSameAs(boundStatement); + verify(preparedStatement).bind(largeValues); + } + + // Test helper classes for Scala Product simulation + static class TestScalaProduct implements scala.Product { + private final Object[] values; + + TestScalaProduct(Object v0, Object v1, Object v2) { + this.values = new Object[] {v0, v1, v2}; + } + + @Override + public int productArity() { + return 3; + } + + @Override + public Object productElement(int n) { + return values[n]; + } + + @Override + public boolean canEqual(Object that) { + return false; + } + } + + static class TestScalaProduct0 implements scala.Product { + @Override + public int productArity() { + return 0; + } + + @Override + public Object productElement(int n) { + throw new IndexOutOfBoundsException(); + } + + @Override + public boolean canEqual(Object that) { + return false; + } + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/QueryParserTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/QueryParserTest.java new file mode 100644 index 0000000..9de7d9e --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/QueryParserTest.java @@ -0,0 +1,617 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.util; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Unit tests for {@link QueryParser}. */ +class QueryParserTest { + + @Test + void testParseBasicInsertQuery() { + String query = + "INSERT INTO analytics.events (user_id, event_time, event_type) VALUES (?, ?, ?)"; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("analytics"); + assertThat(queryInfo.getTableName()).isEqualTo("events"); + assertThat(queryInfo.getColumnNames()) + .containsExactly("user_id", "event_time", "event_type"); + } + + @Test + void testParseBasicUpdateQuery() { + String query = "UPDATE analytics.events SET event_type=?, last_seen=? WHERE user_id=?"; + + QueryParser.UpdateQueryInfo updateInfo = QueryParser.parseUpdateQuery(query); + + assertThat(updateInfo.getKeyspace()).isEqualTo("analytics"); + assertThat(updateInfo.getTableName()).isEqualTo("events"); + assertThat(updateInfo.getSetColumns()).containsExactly("event_type", "last_seen"); + assertThat(updateInfo.getWhereColumns()).containsExactly("user_id"); + } + + @Test + void testParseInsertWithQuotedIdentifiers() { + String query = + "INSERT INTO \"MixedCase\".\"user-profile\" (\"user-id\", \"full name\") VALUES (?, ?)"; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("MixedCase"); + assertThat(queryInfo.getTableName()).isEqualTo("user-profile"); + assertThat(queryInfo.getColumnNames()).containsExactly("user-id", "full name"); + } + + @Test + void testParseUpdateWithQuotedIdentifiers() { + String query = "UPDATE \"keyspace\".\"table\" SET \"mixed-case\"=? WHERE \"id\"=?"; + + QueryParser.UpdateQueryInfo updateInfo = QueryParser.parseUpdateQuery(query); + + assertThat(updateInfo.getKeyspace()).isEqualTo("keyspace"); + assertThat(updateInfo.getTableName()).isEqualTo("table"); + assertThat(updateInfo.getSetColumns()).containsExactly("mixed-case"); + assertThat(updateInfo.getWhereColumns()).containsExactly("id"); + } + + @Test + void testParseInsertWithUsingClause() { + String query = "INSERT INTO ks.table (id, name) VALUES (?, ?) USING TTL 3600"; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("ks"); + assertThat(queryInfo.getTableName()).isEqualTo("table"); + assertThat(queryInfo.getColumnNames()).containsExactly("id", "name"); + } + + @Test + void testParseUpdateWithUsingClause() { + String query = "UPDATE ks.table USING TTL 3600 SET name=? WHERE id=?"; + + QueryParser.UpdateQueryInfo updateInfo = QueryParser.parseUpdateQuery(query); + + assertThat(updateInfo.getKeyspace()).isEqualTo("ks"); + assertThat(updateInfo.getTableName()).isEqualTo("table"); + assertThat(updateInfo.getSetColumns()).containsExactly("name"); + assertThat(updateInfo.getWhereColumns()).containsExactly("id"); + } + + @Test + void testParseQueryWithSemicolon() { + String query = "INSERT INTO ks.table (id) VALUES (?);"; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("ks"); + assertThat(queryInfo.getTableName()).isEqualTo("table"); + assertThat(queryInfo.getColumnNames()).containsExactly("id"); + } + + @Test + void testParseUpdateWithMultipleWhereConditions() { + String query = "UPDATE ks.table SET col1=?, col2=? WHERE pk1=? AND pk2=? AND ck1=?"; + + QueryParser.UpdateQueryInfo updateInfo = QueryParser.parseUpdateQuery(query); + + assertThat(updateInfo.getKeyspace()).isEqualTo("ks"); + assertThat(updateInfo.getTableName()).isEqualTo("table"); + assertThat(updateInfo.getSetColumns()).containsExactly("col1", "col2"); + assertThat(updateInfo.getWhereColumns()).containsExactly("pk1", "pk2", "ck1"); + } + + @Test + void testParseInsertWithIfNotExists() { + String query = "INSERT INTO ks.table (id, name) VALUES (?, ?) IF NOT EXISTS"; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("ks"); + assertThat(queryInfo.getTableName()).isEqualTo("table"); + assertThat(queryInfo.getColumnNames()).containsExactly("id", "name"); + } + + @Test + void testParseUpdateWithIfClause() { + String query = "UPDATE ks.table SET name=? WHERE id=? IF name='old_value'"; + + QueryParser.UpdateQueryInfo updateInfo = QueryParser.parseUpdateQuery(query); + + assertThat(updateInfo.getKeyspace()).isEqualTo("ks"); + assertThat(updateInfo.getTableName()).isEqualTo("table"); + assertThat(updateInfo.getSetColumns()).containsExactly("name"); + assertThat(updateInfo.getWhereColumns()).containsExactly("id"); + } + + @Test + void testParseQueryWithEscapedQuotes() { + String query = "INSERT INTO ks.table (\"col\"\"with\"\"quotes\") VALUES (?)"; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("ks"); + assertThat(queryInfo.getTableName()).isEqualTo("table"); + assertThat(queryInfo.getColumnNames()).containsExactly("col\"with\"quotes"); + } + + // Negative test cases + + @Test + void testParseInsertFailsWithoutKeyspace() { + String query = "INSERT INTO table (id) VALUES (?)"; + + assertThatThrownBy(() -> QueryParser.parseInsertQuery(query)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Static mode requires fully qualified keyspace.table format"); + } + + @Test + void testParseUpdateFailsWithoutKeyspace() { + String query = "UPDATE table SET col=? WHERE id=?"; + + assertThatThrownBy(() -> QueryParser.parseUpdateQuery(query)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Static mode requires fully qualified keyspace.table format"); + } + + @Test + void testParseInsertFailsWithLiterals() { + String query = "INSERT INTO ks.table (id, name) VALUES (123, 'literal')"; + + assertThatThrownBy(() -> QueryParser.parseInsertQuery(query)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Static mode supports only parameter placeholders") + .hasMessageContaining("Found: '123'"); + } + + @Test + void testParseUpdateFailsWithLiteralsInSet() { + String query = "UPDATE ks.table SET name='literal' WHERE id=?"; + + assertThatThrownBy(() -> QueryParser.parseUpdateQuery(query)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Static mode supports only parameter placeholders") + .hasMessageContaining("Found: ''literal''"); + } + + @Test + void testParseUpdateFailsWithLiteralsInWhere() { + String query = "UPDATE ks.table SET name=? WHERE id=123"; + + assertThatThrownBy(() -> QueryParser.parseUpdateQuery(query)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "WHERE condition must contain at least one parameter placeholder") + .hasMessageContaining("Found: 'id=123'"); + } + + @Test + void testParseInsertFailsWithColumnValueCountMismatch() { + String query = + "INSERT INTO ks.table (id, name, email) VALUES (?, ?)"; // 3 columns, 2 values + + assertThatThrownBy(() -> QueryParser.parseInsertQuery(query)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Column count") + .hasMessageContaining("must match value placeholder count"); + } + + @Test + void testParseUpdateSucceedsWithNonEqualityOperator() { + String query = "UPDATE ks.table SET name=? WHERE id > ?"; + + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("name"); + assertThat(info.getWhereColumns()).containsExactly("id"); + } + + @Test + void testParseValidationFailures() { + // INSERT with invalid format - missing parentheses + assertThatThrownBy(() -> QueryParser.parseInsertQuery("INSERT INTO ks.table id VALUES ?")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid INSERT query format"); + + // UPDATE with invalid format - missing SET keyword + assertThatThrownBy(() -> QueryParser.parseUpdateQuery("UPDATE ks.table name=? WHERE id=?")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid UPDATE query format"); + + // Null queries + assertThatThrownBy(() -> QueryParser.parseInsertQuery(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("insertQuery cannot be null or blank"); + + assertThatThrownBy(() -> QueryParser.parseUpdateQuery(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("updateQuery cannot be null or blank"); + + // Blank/empty queries + assertThatThrownBy(() -> QueryParser.parseInsertQuery(" ")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("insertQuery cannot be null or blank"); + + assertThatThrownBy(() -> QueryParser.parseUpdateQuery("")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("updateQuery cannot be null or blank"); + + // INSERT with empty columns + assertThatThrownBy(() -> QueryParser.parseInsertQuery("INSERT INTO ks.table () VALUES ()")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("At least one column must be specified"); + + // UPDATE with empty SET clause + assertThatThrownBy(() -> QueryParser.parseUpdateQuery("UPDATE ks.table SET WHERE id=?")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid UPDATE query format"); + + // UPDATE with empty WHERE clause + assertThatThrownBy(() -> QueryParser.parseUpdateQuery("UPDATE ks.table SET name=? WHERE")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid UPDATE query format"); + } + + // Edge cases + + @Test + void testParseWithExtraWhitespace() { + String query = " INSERT INTO ks.table ( id , name ) VALUES ( ? , ? ) "; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("ks"); + assertThat(queryInfo.getTableName()).isEqualTo("table"); + assertThat(queryInfo.getColumnNames()).containsExactly("id", "name"); + } + + @Test + void testParseCaseInsensitive() { + String query = "insert into ks.table (id, name) values (?, ?)"; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("ks"); + assertThat(queryInfo.getTableName()).isEqualTo("table"); + assertThat(queryInfo.getColumnNames()).containsExactly("id", "name"); + } + + @Test + void testParseUpdateCaseInsensitive() { + String query = "update ks.table set name=? where id=?"; + + QueryParser.UpdateQueryInfo updateInfo = QueryParser.parseUpdateQuery(query); + + assertThat(updateInfo.getKeyspace()).isEqualTo("ks"); + assertThat(updateInfo.getTableName()).isEqualTo("table"); + assertThat(updateInfo.getSetColumns()).containsExactly("name"); + assertThat(updateInfo.getWhereColumns()).containsExactly("id"); + } + + @Test + void testParseComplexRealWorldQuery() { + String query = + "INSERT INTO \"analytics\".\"user_events\" " + + "(\"user_id\", \"event_type\", \"timestamp\", \"properties\") " + + "VALUES (?, ?, ?, ?) " + + "USING TTL 86400 AND TIMESTAMP 1234567890"; + + QueryParser.QueryInfo queryInfo = QueryParser.parseInsertQuery(query); + + assertThat(queryInfo.getKeyspace()).isEqualTo("analytics"); + assertThat(queryInfo.getTableName()).isEqualTo("user_events"); + assertThat(queryInfo.getColumnNames()) + .containsExactly("user_id", "event_type", "timestamp", "properties"); + } + + @Test + void testToStringMethods() { + QueryParser.QueryInfo queryInfo = + new QueryParser.QueryInfo("ks", "table", Arrays.asList("id", "name")); + + // Test getters directly instead of relying on toString + assertThat(queryInfo.getKeyspace()).isEqualTo("ks"); + assertThat(queryInfo.getTableName()).isEqualTo("table"); + assertThat(queryInfo.getColumnNames()).containsExactly("id", "name"); + + // Test that toString() doesn't throw and returns something + String result = queryInfo.toString(); + assertThat(result).isNotNull(); + assertThat(result).isNotEmpty(); + + QueryParser.UpdateQueryInfo updateInfo = + new QueryParser.UpdateQueryInfo( + "ks", "table", Arrays.asList("name"), Arrays.asList("id")); + + // Test getters directly + assertThat(updateInfo.getKeyspace()).isEqualTo("ks"); + assertThat(updateInfo.getTableName()).isEqualTo("table"); + assertThat(updateInfo.getSetColumns()).containsExactly("name"); + assertThat(updateInfo.getWhereColumns()).containsExactly("id"); + + // Test that toString() doesn't throw and returns something + String updateResult = updateInfo.toString(); + assertThat(updateResult).isNotNull(); + assertThat(updateResult).isNotEmpty(); + } + + // Extended tests for various WHERE operators (merged from QueryParserExtendedTest) + + @Test + void testUpdateWithInOperator() { + String query = "UPDATE ks.table SET status=? WHERE id IN (?, ?, ?)"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("status"); + assertThat(info.getWhereColumns()).containsExactly("id"); + } + + @Test + void testUpdateWithGreaterThanOperator() { + String query = "UPDATE ks.table SET flag=? WHERE timestamp > ?"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("flag"); + assertThat(info.getWhereColumns()).containsExactly("timestamp"); + } + + @Test + void testUpdateWithLessThanEqualOperator() { + String query = "UPDATE ks.table SET active=? WHERE age <= ?"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("active"); + assertThat(info.getWhereColumns()).containsExactly("age"); + } + + @Test + void testUpdateWithNotEqualOperator() { + String query = "UPDATE ks.table SET deleted=? WHERE status != ?"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("deleted"); + assertThat(info.getWhereColumns()).containsExactly("status"); + } + + @Test + void testUpdateWithMultipleOperators() { + String query = + "UPDATE ks.table SET flag=?, status=? WHERE id=? AND timestamp > ? AND type IN (?, ?)"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("flag", "status"); + assertThat(info.getWhereColumns()).containsExactlyInAnyOrder("id", "timestamp", "type"); + } + + @Test + void testUpdateWithIfExistsAndInOperator() { + String query = "UPDATE ks.table SET status=? WHERE id IN (?, ?, ?) IF EXISTS"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("status"); + assertThat(info.getWhereColumns()).containsExactly("id"); + } + + @Test + void testUpdateWithUsingTTLAndInOperator() { + String query = "UPDATE ks.table USING TTL 3600 SET status=? WHERE id IN (?, ?)"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("status"); + assertThat(info.getWhereColumns()).containsExactly("id"); + } + + @Test + void testUpdateWithComplexIfCondition() { + String query = + "UPDATE ks.table SET status=? WHERE id=? IF status != 'deleted' AND timestamp > 1000"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("status"); + assertThat(info.getWhereColumns()).containsExactly("id"); + } + + @Test + void testUpdateWithQuotedColumnAndInOperator() { + String query = "UPDATE ks.table SET \"Status\"=? WHERE \"User-ID\" IN (?, ?, ?)"; + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(query); + + assertThat(info.getKeyspace()).isEqualTo("ks"); + assertThat(info.getTableName()).isEqualTo("table"); + assertThat(info.getSetColumns()).containsExactly("Status"); + assertThat(info.getWhereColumns()).containsExactly("User-ID"); + } + + // Comprehensive INSERT query tests + + @Test + void testInsertQueryComprehensive() { + // Test with newlines and whitespace + QueryParser.QueryInfo info1 = + QueryParser.parseInsertQuery( + "INSERT INTO\n\t ks.tbl\n (id,\n name)\n VALUES\n (?,\n ?)"); + assertThat(info1.getKeyspace()).isEqualTo("ks"); + assertThat(info1.getTableName()).isEqualTo("tbl"); + assertThat(info1.getColumnNames()).containsExactly("id", "name"); + + // Test with dollar signs in identifiers + QueryParser.QueryInfo info2 = + QueryParser.parseInsertQuery( + "INSERT INTO keyspace1.table1 (user_$id, name$value) VALUES (?, ?)"); + assertThat(info2.getKeyspace()).isEqualTo("keyspace1"); + assertThat(info2.getTableName()).isEqualTo("table1"); + assertThat(info2.getColumnNames()).containsExactly("user_$id", "name$value"); + + // Test with escaped quotes in identifiers + QueryParser.QueryInfo info3 = + QueryParser.parseInsertQuery( + "INSERT INTO ks2.tbl2 (\"na\"\"me\", \"val\"\"ue\") VALUES (?, ?)"); + assertThat(info3.getKeyspace()).isEqualTo("ks2"); + assertThat(info3.getTableName()).isEqualTo("tbl2"); + assertThat(info3.getColumnNames()).containsExactly("na\"me", "val\"ue"); + + // Test with multiple USING clauses (TTL and TIMESTAMP) + QueryParser.QueryInfo info4 = + QueryParser.parseInsertQuery( + "INSERT INTO ks3.tbl3 (id) VALUES (?) USING TTL 3600 AND TIMESTAMP 123456789"); + assertThat(info4.getKeyspace()).isEqualTo("ks3"); + assertThat(info4.getTableName()).isEqualTo("tbl3"); + assertThat(info4.getColumnNames()).containsExactly("id"); + + // Test with Unicode identifiers + QueryParser.QueryInfo info5 = + QueryParser.parseInsertQuery( + "INSERT INTO unicode_ks.unicode_tbl (用户名, 电子邮件) VALUES (?, ?)"); + assertThat(info5.getKeyspace()).isEqualTo("unicode_ks"); + assertThat(info5.getTableName()).isEqualTo("unicode_tbl"); + assertThat(info5.getColumnNames()).containsExactly("用户名", "电子邮件"); + + // Test error case: mixed placeholders and literals + assertThatThrownBy( + () -> + QueryParser.parseInsertQuery( + "INSERT INTO ks.tbl (id, age, name) VALUES (?, 42, ?)")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("only parameter placeholders (?)") + .hasMessageContaining("42"); + } + + // Comprehensive UPDATE query tests + + @Test + void testUpdateQueryComprehensive() { + // Test all comparison operators + String[] operatorQueries = { + "UPDATE ks1.tbl1 SET val=? WHERE id = ?", + "UPDATE ks2.tbl2 SET val=? WHERE id != ?", + "UPDATE ks3.tbl3 SET val=? WHERE id < ?", + "UPDATE ks4.tbl4 SET val=? WHERE id > ?", + "UPDATE ks5.tbl5 SET val=? WHERE id <= ?", + "UPDATE ks6.tbl6 SET val=? WHERE id >= ?" + }; + String[] expectedKeyspaces = {"ks1", "ks2", "ks3", "ks4", "ks5", "ks6"}; + String[] expectedTables = {"tbl1", "tbl2", "tbl3", "tbl4", "tbl5", "tbl6"}; + + for (int i = 0; i < operatorQueries.length; i++) { + QueryParser.UpdateQueryInfo info = QueryParser.parseUpdateQuery(operatorQueries[i]); + assertThat(info.getKeyspace()).isEqualTo(expectedKeyspaces[i]); + assertThat(info.getTableName()).isEqualTo(expectedTables[i]); + assertThat(info.getSetColumns()).containsExactly("val"); + assertThat(info.getWhereColumns()).containsExactly("id"); + } + + // Test IN operator with multiple placeholders + QueryParser.UpdateQueryInfo info1 = + QueryParser.parseUpdateQuery( + "UPDATE in_ks.in_tbl SET status=? WHERE id IN (?, ?, ?)"); + assertThat(info1.getKeyspace()).isEqualTo("in_ks"); + assertThat(info1.getTableName()).isEqualTo("in_tbl"); + assertThat(info1.getSetColumns()).containsExactly("status"); + assertThat(info1.getWhereColumns()).containsExactly("id"); + + // Test complex WHERE conditions with AND/OR + // Note: OR splits conditions differently, so we only get columns before OR + QueryParser.UpdateQueryInfo info2 = + QueryParser.parseUpdateQuery( + "UPDATE complex_ks.complex_tbl SET val=? WHERE id=? AND status IN(?,?) AND age> ?"); + assertThat(info2.getKeyspace()).isEqualTo("complex_ks"); + assertThat(info2.getTableName()).isEqualTo("complex_tbl"); + assertThat(info2.getSetColumns()).containsExactly("val"); + assertThat(info2.getWhereColumns()).containsExactly("id", "status", "age"); + + // Test USING clause after table name + QueryParser.UpdateQueryInfo info3 = + QueryParser.parseUpdateQuery( + "UPDATE using_ks.using_tbl USING TTL 3600 AND TIMESTAMP 1 SET a=? WHERE id=?"); + assertThat(info3.getKeyspace()).isEqualTo("using_ks"); + assertThat(info3.getTableName()).isEqualTo("using_tbl"); + assertThat(info3.getSetColumns()).containsExactly("a"); + assertThat(info3.getWhereColumns()).containsExactly("id"); + + // Test quoted identifiers with special characters + QueryParser.UpdateQueryInfo info5 = + QueryParser.parseUpdateQuery( + "UPDATE quoted_ks.quoted_tbl SET \"A-Name\" = ?, \"B\"\"Q\" = ? WHERE \"C-Id\" = ?"); + assertThat(info5.getKeyspace()).isEqualTo("quoted_ks"); + assertThat(info5.getTableName()).isEqualTo("quoted_tbl"); + assertThat(info5.getSetColumns()).containsExactly("A-Name", "B\"Q"); + assertThat(info5.getWhereColumns()).containsExactly("C-Id"); + } + + @Test + void testSpecialCasesAndEdgeCases() { + // Test very long column list (100 columns) + StringBuilder query = new StringBuilder("INSERT INTO long_ks.long_tbl ("); + StringBuilder values = new StringBuilder(" VALUES ("); + for (int i = 0; i < 100; i++) { + if (i > 0) { + query.append(", "); + values.append(", "); + } + query.append("col").append(i); + values.append("?"); + } + query.append(")").append(values).append(")"); + + QueryParser.QueryInfo longInfo = QueryParser.parseInsertQuery(query.toString()); + assertThat(longInfo.getKeyspace()).isEqualTo("long_ks"); + assertThat(longInfo.getTableName()).isEqualTo("long_tbl"); + assertThat(longInfo.getColumnNames()).hasSize(100); + assertThat(longInfo.getColumnNames().get(0)).isEqualTo("col0"); + assertThat(longInfo.getColumnNames().get(99)).isEqualTo("col99"); + + // Test empty SET clause detection + assertThatThrownBy(() -> QueryParser.parseUpdateQuery("UPDATE ks.tbl SET WHERE id=?")) + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> QueryParser.parseUpdateQuery("UPDATE ks.tbl SET WHERE id=?")) + .isInstanceOf(IllegalArgumentException.class); + + // Test missing keyspace with quoted table + assertThatThrownBy( + () -> + QueryParser.parseInsertQuery( + "INSERT INTO \"MyTable\" (id) VALUES (?)")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("fully qualified keyspace.table"); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/RecordWriterFactoryTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/RecordWriterFactoryTest.java new file mode 100644 index 0000000..a49d44e --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/util/RecordWriterFactoryTest.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.util; + +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.connector.cassandra.sink.config.CassandraSinkConfig; +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.writer.CassandraRecordWriter; +import org.apache.flink.connector.cassandra.sink.writer.CqlRecordWriter; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.types.Row; + +import com.datastax.driver.core.Cluster; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link RecordWriterFactory}. */ +public class RecordWriterFactoryTest { + + @Mock private ClusterBuilder clusterBuilder; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + // Setup clusterBuilder to return a valid cluster + when(clusterBuilder.getCluster()).thenReturn(mock(Cluster.class)); + } + + @Test + void testVariousFormatsReturnCqlRecordWriter() { + // Test ROW format returns CqlRecordWriter + CqlSinkConfig rowConfig = + CqlSinkConfig.forRow().withQuery("INSERT INTO ks.tbl (id) VALUES (?)"); + CassandraRecordWriter rowWriter = + RecordWriterFactory.create(clusterBuilder, rowConfig); + assertThat(rowWriter).isInstanceOf(CqlRecordWriter.class); + assertThat(rowWriter).isExactlyInstanceOf(CqlRecordWriter.class); + + // Test TUPLE format returns CqlRecordWriter + CqlSinkConfig tupleConfig = + CqlSinkConfig.forTuple().withQuery("INSERT INTO ks.tbl (id) VALUES (?)"); + CassandraRecordWriter tupleWriter = + RecordWriterFactory.create(clusterBuilder, tupleConfig); + assertThat(tupleWriter).isInstanceOf(CqlRecordWriter.class); + assertThat(tupleWriter).isExactlyInstanceOf(CqlRecordWriter.class); + + // Test SCALA_PRODUCT format returns CqlRecordWriter + CqlSinkConfig scalaConfig = + CqlSinkConfig.forScalaProduct().withQuery("INSERT INTO ks.tbl (id) VALUES (?)"); + CassandraRecordWriter scalaWriter = + RecordWriterFactory.create(clusterBuilder, scalaConfig); + assertThat(scalaWriter).isInstanceOf(CqlRecordWriter.class); + assertThat(scalaWriter).isExactlyInstanceOf(CqlRecordWriter.class); + } + + @Test + void testUnsupportedFormatTypeThrowsNullPointerException() { + // Create a mock config that returns null format type + CassandraSinkConfig config = mock(CassandraSinkConfig.class); + when(config.getRecordFormatType()).thenReturn(null); + + // The factory will throw NPE when trying to switch on null + assertThatThrownBy(() -> RecordWriterFactory.create(clusterBuilder, config)) + .isInstanceOf(NullPointerException.class); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/AbstractRecordWriterTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/AbstractRecordWriterTest.java new file mode 100644 index 0000000..d6fadf4 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/AbstractRecordWriterTest.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.writer; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.ResultSetFuture; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.google.common.util.concurrent.ListenableFuture; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.InOrder; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link AbstractRecordWriter}. */ +public class AbstractRecordWriterTest { + + @Mock private Session session; + @Mock private Cluster cluster; + @Mock private Statement statement; + @Mock private ResultSetFuture future; + + private TestAbstractRecordWriter writer; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + } + + @Test + void testExecuteStatementDelegatesToSessionExecuteAsync() { + writer = new TestAbstractRecordWriter(session, cluster); + when(session.executeAsync(statement)).thenReturn(future); + ListenableFuture result = writer.executeStatement(statement); + assertThat(result).isSameAs(future); + verify(session, times(1)).executeAsync(statement); + verifyNoMoreInteractions(session); + } + + @Test + void testCloseComprehensive() throws Exception { + // Test normal close with both session and cluster + writer = new TestAbstractRecordWriter(session, cluster); + writer.close(); + InOrder inOrder = inOrder(session, cluster); + inOrder.verify(session).close(); + inOrder.verify(cluster).close(); + + // Reset mocks for next test + MockitoAnnotations.openMocks(this); + + // Test idempotency - close() can be called multiple times + writer = new TestAbstractRecordWriter(session, cluster); + writer.close(); + writer.close(); + verify(session, atLeastOnce()).close(); + verify(cluster, atLeastOnce()).close(); + + // Reset mocks for next test + MockitoAnnotations.openMocks(this); + + // Test with null session - only cluster should be closed + writer = new TestAbstractRecordWriter(null, cluster); + writer.close(); + verify(cluster).close(); + verifyNoInteractions(session); + + // Reset mocks for next test + MockitoAnnotations.openMocks(this); + + // Test with null cluster - only session should be closed + writer = new TestAbstractRecordWriter(session, null); + writer.close(); + verify(session).close(); + verifyNoInteractions(cluster); + + // Reset mocks for next test + MockitoAnnotations.openMocks(this); + + // Test with both null - should not throw + writer = new TestAbstractRecordWriter(null, null); + writer.close(); + verifyNoInteractions(session, cluster); + } + + @Test + void testCloseWhenSessionCloseThrows() { + writer = new TestAbstractRecordWriter(session, cluster); + RuntimeException sessionError = new RuntimeException("Session close failed"); + doThrow(sessionError).when(session).close(); + + // The exception should be thrown, cluster.close() won't be reached + assertThatThrownBy(() -> writer.close()) + .isInstanceOf(RuntimeException.class) + .hasMessage("Session close failed"); + + verify(session).close(); + verify(cluster, never()).close(); + } + + @Test + void testCloseWhenClusterCloseThrows() { + writer = new TestAbstractRecordWriter(session, cluster); + RuntimeException clusterError = new RuntimeException("Cluster close failed"); + doThrow(clusterError).when(cluster).close(); + + // Session closes successfully, then cluster.close() throws + assertThatThrownBy(() -> writer.close()) + .isInstanceOf(RuntimeException.class) + .hasMessage("Cluster close failed"); + + verify(session).close(); + verify(cluster).close(); + } + + @Test + void testGetSessionAndCluster() { + writer = new TestAbstractRecordWriter(session, cluster); + assertThat(writer.getSession()).isSameAs(session); + assertThat(writer.getCluster()).isSameAs(cluster); + writer = new TestAbstractRecordWriter(null, null); + assertThat(writer.getSession()).isNull(); + assertThat(writer.getCluster()).isNull(); + } + + // Test implementation of AbstractRecordWriter + static class TestAbstractRecordWriter extends AbstractRecordWriter { + private final Session session; + private final Cluster cluster; + + TestAbstractRecordWriter(Session session, Cluster cluster) { + this.session = session; + this.cluster = cluster; + } + + @Override + public Statement prepareStatement(String input) { + throw new UnsupportedOperationException("Not needed for AbstractRecordWriter tests"); + } + + @Override + public Session getSession() { + return session; + } + + @Override + protected Cluster getCluster() { + return cluster; + } + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/CqlRecordWriterTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/CqlRecordWriterTest.java new file mode 100644 index 0000000..13f9def --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/CqlRecordWriterTest.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.writer; + +import org.apache.flink.connector.cassandra.sink.config.CqlSinkConfig; +import org.apache.flink.connector.cassandra.sink.planner.core.components.StatementPlanner; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.types.Row; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.ResultSetFuture; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.google.common.util.concurrent.ListenableFuture; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link CqlRecordWriter}. */ +public class CqlRecordWriterTest { + + @Mock private ClusterBuilder clusterBuilder; + @Mock private CqlSinkConfig config; + @Mock private Cluster cluster; + @Mock private Session session; + @Mock private StatementPlanner planner; + @Mock private Statement statement; + @Mock private ResultSetFuture future; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + } + + @Test + void testSuccessfulClusterCreationAndConnection() { + // Execute - using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Verify + assertThat(writer).isNotNull(); + assertThat(writer.getSession()).isSameAs(session); + } + + @Test + void testGetClusterFailsThrowsRuntimeException() { + // Setup + RuntimeException clusterError = new RuntimeException("Cluster build failed"); + when(clusterBuilder.getCluster()).thenThrow(clusterError); + + // Execute & Verify + assertThatThrownBy(() -> new CqlRecordWriter<>(clusterBuilder, config)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Failed to create Cassandra cluster from ClusterBuilder") + .hasCause(clusterError); + } + + @Test + void testConnectFailsThrowsRuntimeExceptionWithCleanup() { + // Setup + RuntimeException connectError = new RuntimeException("Connect failed"); + when(clusterBuilder.getCluster()).thenReturn(cluster); + when(cluster.connect()).thenThrow(connectError); + + // Execute & Verify + assertThatThrownBy(() -> new CqlRecordWriter<>(clusterBuilder, config)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Failed to connect to Cassandra cluster") + .hasCause(connectError); + + // Verify cleanup + verify(cluster).close(); + } + + @Test + void testPrepareStatementDelegatesToPlannerPlan() throws Exception { + // Setup writer using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Setup test + Row input = Row.of(1, "test"); + when(planner.plan(input, session, config)).thenReturn(statement); + + // Execute + Statement result = writer.prepareStatement(input); + + // Verify + assertThat(result).isSameAs(statement); + verify(planner).plan(input, session, config); + } + + @Test + void testPrepareStatementPlannerThrowsExceptionPropagated() throws Exception { + // Setup writer using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Setup test + Row input = Row.of(1, "test"); + RuntimeException plannerError = new RuntimeException("Planning failed"); + when(planner.plan(input, session, config)).thenThrow(plannerError); + + // Execute & Verify + assertThatThrownBy(() -> writer.prepareStatement(input)).isSameAs(plannerError); + } + + @Test + void testGetSessionReturnsSessionFromClusterConnect() { + // Execute - using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Verify + assertThat(writer.getSession()).isSameAs(session); + } + + @Test + void testCloseCallsSuperClose() throws Exception { + // Setup writer using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Execute + writer.close(); + + // Verify + verify(planner).close(); + verify(session).close(); + } + + @Test + void testCloseIdempotency() throws Exception { + // Setup writer using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Execute + writer.close(); + writer.close(); + + // Verify idempotent close + verify(planner, atLeastOnce()).close(); + verify(session, atLeastOnce()).close(); + } + + @Test + void testCompleteWriteFlowPlanToExecuteAsyncToFuture() throws Exception { + // Setup writer using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Setup test + Row input = Row.of(1, "test"); + when(planner.plan(input, session, config)).thenReturn(statement); + when(session.executeAsync(any(Statement.class))).thenReturn(future); + + // Execute + ListenableFuture result = writer.write(input); + + // Verify + assertThat(result).isSameAs(future); + verify(planner).plan(input, session, config); + verify(session).executeAsync(statement); + } + + @Test + void testWritePassesPlannerStatementUnchanged() throws Exception { + // Setup writer using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Setup test - planner returns a statement + Row input = Row.of(1, "test"); + when(planner.plan(input, session, config)).thenReturn(statement); + when(session.executeAsync(statement)).thenReturn(future); + + // Execute + writer.write(input); + + // Verify exact statement instance passed to executeAsync + verify(session).executeAsync(statement); + } + + @Test + void testResourceSafetyWhenPrepareStatementFails() throws Exception { + // Setup writer using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Setup test + Row input = Row.of(1, "test"); + RuntimeException prepareError = new RuntimeException("Prepare failed"); + when(planner.plan(input, session, config)).thenThrow(prepareError); + + // Execute + assertThatThrownBy(() -> writer.write(input)).isSameAs(prepareError); + + // Verify no executeAsync called + verify(session, never()).executeAsync(any(Statement.class)); + + // Verify close still works after failure + writer.close(); + verify(planner).close(); + verify(session).close(); + } + + @Test + void testResourceSafetyWhenExecuteAsyncFails() throws Exception { + // Setup writer using @VisibleForTesting constructor + CqlRecordWriter writer = new CqlRecordWriter<>(session, planner, config); + + // Setup test + Row input = Row.of(1, "test"); + RuntimeException executeError = new RuntimeException("Execute failed"); + when(planner.plan(input, session, config)).thenReturn(statement); + when(session.executeAsync(statement)).thenThrow(executeError); + + // Execute + assertThatThrownBy(() -> writer.write(input)).isSameAs(executeError); + + // Verify close still works after failure + writer.close(); + verify(planner).close(); + verify(session).close(); + } +} diff --git a/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/PojoRecordWriterTest.java b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/PojoRecordWriterTest.java new file mode 100644 index 0000000..ab4e501 --- /dev/null +++ b/flink-connector-cassandra/src/test/java/org/apache/flink/connector/cassandra/sink/writer/PojoRecordWriterTest.java @@ -0,0 +1,292 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.cassandra.sink.writer; + +import org.apache.flink.connector.cassandra.sink.config.PojoSinkConfig; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.ResultSetFuture; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.datastax.driver.mapping.Mapper; +import com.datastax.driver.mapping.annotations.Column; +import com.datastax.driver.mapping.annotations.PartitionKey; +import com.datastax.driver.mapping.annotations.Table; +import com.google.common.util.concurrent.ListenableFuture; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.InOrder; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +/** Unit tests for {@link PojoRecordWriter}. */ +public class PojoRecordWriterTest { + + /** Test POJO with Cassandra annotations. */ + @Table(keyspace = "test_keyspace", name = "users") + public static class User { + @PartitionKey + @Column(name = "user_id") + private UUID userId; + + @Column(name = "username") + private String username; + + @Column(name = "email") + private String email; + + @Column(name = "age") + private Integer age; + + public User() {} + + public User(UUID userId, String username, String email, Integer age) { + this.userId = userId; + this.username = username; + this.email = email; + this.age = age; + } + + public UUID getUserId() { + return userId; + } + + public void setUserId(UUID userId) { + this.userId = userId; + } + + public String getUsername() { + return username; + } + + public void setUsername(String username) { + this.username = username; + } + + public String getEmail() { + return email; + } + + public void setEmail(String email) { + this.email = email; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + } + + @Mock private ClusterBuilder clusterBuilder; + @Mock private PojoSinkConfig config; + @Mock private Cluster cluster; + @Mock private Session session; + @Mock private Mapper mapper; + @Mock private Statement statement; + @Mock private ResultSetFuture future; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + } + + @Test + void testConstructorValidationRejectsNullParameters() { + // Test null ClusterBuilder rejection + assertThatThrownBy(() -> new PojoRecordWriter<>(null, config)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("ClusterBuilder cannot be null"); + + // Test null PojoSinkConfig rejection + assertThatThrownBy(() -> new PojoRecordWriter<>(clusterBuilder, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("PojoSinkConfig cannot be null"); + } + + @Test + void testConstructorCallsBuilderAndClusterMethods() { + String keyspace = "test_keyspace"; + when(config.getKeyspace()).thenReturn(keyspace); + when(config.getPojoClass()).thenReturn(User.class); + when(config.getMapperOptions()).thenReturn(null); + + // Test getCluster() failure + RuntimeException clusterError = new RuntimeException("Cluster creation failed"); + when(clusterBuilder.getCluster()).thenThrow(clusterError); + + assertThatThrownBy(() -> new PojoRecordWriter<>(clusterBuilder, config)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Failed to initialize PojoRecordWriter") + .hasMessageContaining(keyspace) + .hasMessageContaining(User.class.getName()) + .hasCause(clusterError); + + // Verify builder.getCluster() was called exactly once + verify(clusterBuilder, times(1)).getCluster(); + verify(cluster, never()).connect(any()); + + // Reset and test connect() failure + MockitoAnnotations.openMocks(this); + when(config.getKeyspace()).thenReturn(keyspace); + when(config.getPojoClass()).thenReturn(User.class); + when(clusterBuilder.getCluster()).thenReturn(cluster); + + RuntimeException connectError = new RuntimeException("Connection failed"); + when(cluster.connect(keyspace)).thenThrow(connectError); + + assertThatThrownBy(() -> new PojoRecordWriter<>(clusterBuilder, config)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Failed to initialize PojoRecordWriter") + .hasMessageContaining(keyspace) + .hasMessageContaining(User.class.getName()) + .hasCause(connectError); + + // Verify cluster.connect() was called with correct keyspace + verify(clusterBuilder, times(1)).getCluster(); + verify(cluster, times(1)).connect(keyspace); + } + + @Test + void testGetSessionAndGetCluster() { + // Create writer using test constructor + PojoRecordWriter writer = new PojoRecordWriter<>(cluster, session, mapper); + + // Verify getSession returns the same Session instance + assertThat(writer.getSession()).isSameAs(session); + + // Verify getCluster returns the same Cluster instance + assertThat(writer.getCluster()).isSameAs(cluster); + } + + @Test + void testPrepareStatementDelegatesToMapper() { + PojoRecordWriter writer = new PojoRecordWriter<>(cluster, session, mapper); + + // Test with valid user + User user = new User(UUID.randomUUID(), "alice", "alice@example.com", 30); + when(mapper.saveQuery(user)).thenReturn(statement); + + Statement result = writer.prepareStatement(user); + + assertThat(result).isSameAs(statement); + verify(mapper, times(1)).saveQuery(user); + verifyNoMoreInteractions(mapper); + + // Test with null input - propagates mapper exception + when(mapper.saveQuery(null)).thenThrow(new NullPointerException("Cannot save null")); + + assertThatThrownBy(() -> writer.prepareStatement(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Cannot save null"); + + verify(mapper, times(1)).saveQuery(null); + } + + @Test + void testExecuteStatementDelegatesToSession() { + PojoRecordWriter writer = new PojoRecordWriter<>(cluster, session, mapper); + + // Test executeStatement delegates to session.executeAsync + when(session.executeAsync(statement)).thenReturn(future); + + ListenableFuture result = writer.executeStatement(statement); + + assertThat(result).isSameAs(future); + verify(session, times(1)).executeAsync(statement); + verifyNoMoreInteractions(session); + + // Test with null statement - propagates session exception + when(session.executeAsync((Statement) null)) + .thenThrow(new IllegalArgumentException("Statement is null")); + + assertThatThrownBy(() -> writer.executeStatement(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Statement is null"); + + verify(session, times(1)).executeAsync((Statement) null); + } + + @Test + void testWriteMethodCallsPrepareStatementThenExecuteStatement() throws Exception { + PojoRecordWriter writer = spy(new PojoRecordWriter<>(cluster, session, mapper)); + + User user = new User(UUID.randomUUID(), "bob", "bob@example.com", 25); + when(mapper.saveQuery(user)).thenReturn(statement); + when(session.executeAsync(statement)).thenReturn(future); + + ListenableFuture result = writer.write(user); + + assertThat(result).isSameAs(future); + + // Verify interaction order + InOrder inOrder = inOrder(writer, mapper, session); + inOrder.verify(writer).prepareStatement(user); + inOrder.verify(mapper).saveQuery(user); + inOrder.verify(writer).executeStatement(statement); + inOrder.verify(session).executeAsync(statement); + inOrder.verifyNoMoreInteractions(); + } + + @Test + void testCloseMethodClosesSessionThenCluster() throws Exception { + PojoRecordWriter writer = new PojoRecordWriter<>(cluster, session, mapper); + + // First close + writer.close(); + + // Verify close order + InOrder closeOrder = inOrder(session, cluster); + closeOrder.verify(session).close(); + closeOrder.verify(cluster).close(); + + // Test idempotency - multiple calls don't throw + writer.close(); // Second call + writer.close(); // Third call + + // Verify close was called at least once (idempotent) + verify(session, atLeastOnce()).close(); + verify(cluster, atLeastOnce()).close(); + } + + @Test + void testNoUnexpectedInteractionsWithMocks() { + PojoRecordWriter writer = new PojoRecordWriter<>(cluster, session, mapper); + assertThat(writer).isNotNull(); + verifyNoMoreInteractions(session, cluster, mapper); + } +}