zentol commented on code in PR #19680:
URL: https://github.com/apache/flink/pull/19680#discussion_r876793907
##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraPojoOutputFormat.java:
##########
@@ -90,67 +83,20 @@ public void open(int taskNumber, int numTasks) {
mapper.setDefaultSaveOptions(optionsArray);
}
}
- this.callback =
- new FutureCallback<Void>() {
- @Override
- public void onSuccess(Void ignored) {
- onWriteSuccess();
- }
-
- @Override
- public void onFailure(Throwable t) {
- onWriteFailure(t);
- }
- };
}
@Override
- public void writeRecord(OUT record) throws IOException {
- if (exception != null) {
- throw new IOException("write record failed", exception);
- }
-
- ListenableFuture<Void> result = mapper.saveAsync(record);
- Futures.addCallback(result, callback);
- }
-
- /**
- * Callback that is invoked after a record is written to Cassandra
successfully.
- *
- * <p>Subclass can override to provide its own logic.
- */
- protected void onWriteSuccess() {}
-
- /**
- * Callback that is invoked when failing to write to Cassandra. Current
implementation will
- * record the exception and fail the job upon next record.
- *
- * <p>Subclass can override to provide its own failure handling logic.
- *
- * @param t the exception
- */
- protected void onWriteFailure(Throwable t) {
- exception = t;
+ public ListenableFuture<Void> send(OUT record) {
Review Comment:
```suggestion
protected ListenableFuture<Void> send(OUT record) {
```
##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
package org.apache.flink.batch.connectors.cassandra;
+import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.io.RichOutputFormat;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
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.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
/**
- * CassandraOutputFormatBase is the common abstract class for writing into
Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into
Apache Cassandra using
+ * output formats.
*
* @param <OUT> Type of the elements to write.
*/
-public abstract class CassandraOutputFormatBase<OUT> extends
RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends
RichOutputFormat<OUT> {
private static final Logger LOG =
LoggerFactory.getLogger(CassandraOutputFormatBase.class);
- private final String insertQuery;
private final ClusterBuilder builder;
+ private Semaphore semaphore;
+ private Duration maxConcurrentRequestsTimeout =
Duration.ofMillis(Long.MAX_VALUE);
+ private int maxConcurrentRequests = Integer.MAX_VALUE;
private transient Cluster cluster;
- private transient Session session;
- private transient PreparedStatement prepared;
- private transient FutureCallback<ResultSet> callback;
- private transient Throwable exception = null;
-
- public CassandraOutputFormatBase(String insertQuery, ClusterBuilder
builder) {
- Preconditions.checkArgument(
- !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or
empty");
+ protected transient Session session;
+ private transient FutureCallback<V> callback;
+ private AtomicReference<Throwable> throwable;
+
+ public CassandraOutputFormatBase(
+ ClusterBuilder builder,
+ int maxConcurrentRequests,
+ Duration maxConcurrentRequestsTimeout) {
Preconditions.checkNotNull(builder, "Builder cannot be null");
-
- this.insertQuery = insertQuery;
this.builder = builder;
+ Preconditions.checkArgument(
+ maxConcurrentRequests > 0, "Max concurrent requests is
expected to be positive");
+ this.maxConcurrentRequests = maxConcurrentRequests;
+ Preconditions.checkNotNull(
+ maxConcurrentRequestsTimeout, "Max concurrent requests timeout
cannot be null");
+ Preconditions.checkArgument(
+ !maxConcurrentRequestsTimeout.isNegative(),
+ "Max concurrent requests timeout is expected to be positive");
+ this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
}
+ /** Configure the connection to Cassandra. */
@Override
public void configure(Configuration parameters) {
this.cluster = builder.getCluster();
}
- /**
- * Opens a Session to Cassandra and initializes the prepared statement.
- *
- * @param taskNumber The number of the parallel instance.
- * @throws IOException Thrown, if the output could not be opened due to an
I/O problem.
- */
+ /** Opens a Session to Cassandra . */
@Override
public void open(int taskNumber, int numTasks) throws IOException {
+ throwable = new AtomicReference<>();
+ this.semaphore = new Semaphore(maxConcurrentRequests);
this.session = cluster.connect();
- this.prepared = session.prepare(insertQuery);
this.callback =
- new FutureCallback<ResultSet>() {
+ new FutureCallback<V>() {
@Override
- public void onSuccess(ResultSet ignored) {
- onWriteSuccess(ignored);
+ public void onSuccess(V ignored) {
+ semaphore.release();
}
@Override
public void onFailure(Throwable t) {
- onWriteFailure(t);
+ throwable.compareAndSet(null, t);
+ LOG.error("Error while writing value.", t);
+ semaphore.release();
}
};
}
+ private void flush() {
+ tryAcquire(maxConcurrentRequests);
+ semaphore.release(maxConcurrentRequests);
+ }
+
+ private void tryAcquire(int permits) {
+ try {
+ SinkUtils.tryAcquire(
+ permits, maxConcurrentRequests,
maxConcurrentRequestsTimeout, semaphore);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
Review Comment:
Have you thought about throwing an IOException?
##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+ private static final long DEFAULT_TEST_TIMEOUT = 5000;
+ private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+ Duration.ofMillis(Long.MAX_VALUE);
+
+ @Test(expected = NoHostAvailableException.class)
+ public void testHostNotFoundErrorHandling() throws Exception {
+ CassandraOutputFormatBase cassandraOutputFormatBase =
+ new CassandraOutputFormatBase(
Review Comment:
avoid raw types
##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
package org.apache.flink.batch.connectors.cassandra;
+import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.io.RichOutputFormat;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
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.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
/**
- * CassandraOutputFormatBase is the common abstract class for writing into
Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into
Apache Cassandra using
+ * output formats.
*
* @param <OUT> Type of the elements to write.
*/
-public abstract class CassandraOutputFormatBase<OUT> extends
RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends
RichOutputFormat<OUT> {
private static final Logger LOG =
LoggerFactory.getLogger(CassandraOutputFormatBase.class);
- private final String insertQuery;
private final ClusterBuilder builder;
+ private Semaphore semaphore;
+ private Duration maxConcurrentRequestsTimeout =
Duration.ofMillis(Long.MAX_VALUE);
+ private int maxConcurrentRequests = Integer.MAX_VALUE;
private transient Cluster cluster;
- private transient Session session;
- private transient PreparedStatement prepared;
- private transient FutureCallback<ResultSet> callback;
- private transient Throwable exception = null;
-
- public CassandraOutputFormatBase(String insertQuery, ClusterBuilder
builder) {
- Preconditions.checkArgument(
- !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or
empty");
+ protected transient Session session;
+ private transient FutureCallback<V> callback;
+ private AtomicReference<Throwable> throwable;
+
+ public CassandraOutputFormatBase(
+ ClusterBuilder builder,
+ int maxConcurrentRequests,
+ Duration maxConcurrentRequestsTimeout) {
Preconditions.checkNotNull(builder, "Builder cannot be null");
-
- this.insertQuery = insertQuery;
this.builder = builder;
+ Preconditions.checkArgument(
+ maxConcurrentRequests > 0, "Max concurrent requests is
expected to be positive");
+ this.maxConcurrentRequests = maxConcurrentRequests;
+ Preconditions.checkNotNull(
+ maxConcurrentRequestsTimeout, "Max concurrent requests timeout
cannot be null");
+ Preconditions.checkArgument(
+ !maxConcurrentRequestsTimeout.isNegative(),
+ "Max concurrent requests timeout is expected to be positive");
+ this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
}
+ /** Configure the connection to Cassandra. */
@Override
public void configure(Configuration parameters) {
this.cluster = builder.getCluster();
}
- /**
- * Opens a Session to Cassandra and initializes the prepared statement.
- *
- * @param taskNumber The number of the parallel instance.
- * @throws IOException Thrown, if the output could not be opened due to an
I/O problem.
- */
+ /** Opens a Session to Cassandra . */
@Override
public void open(int taskNumber, int numTasks) throws IOException {
Review Comment:
Does this actually throw an IOException? Doesn't seem so; if we remove this
we could skip the signature change in the pojo output format. (changing it here
instead of the pojo format is preferable since the latter is more user-facing
than this one)
##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing
into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+abstract class CassandraColumnarOutputFormatBase<OUT>
+ extends CassandraOutputFormatBase<OUT, ResultSet> {
+ private final String insertQuery;
+ private transient PreparedStatement prepared;
+
+ public CassandraColumnarOutputFormatBase(
+ String insertQuery,
+ ClusterBuilder builder,
+ int maxConcurrentRequests,
+ Duration maxConcurrentRequestsTimeout) {
+ super(builder, maxConcurrentRequests, maxConcurrentRequestsTimeout);
+ Preconditions.checkArgument(
+ !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or
empty");
+ this.insertQuery = insertQuery;
+ }
+
+ /** Opens a Session to Cassandra and initializes the prepared statement. */
+ @Override
+ public void open(int taskNumber, int numTasks) throws IOException {
+ super.open(taskNumber, numTasks);
+ this.prepared = session.prepare(insertQuery);
+ }
+
+ @Override
+ public ListenableFuture<ResultSet> send(OUT record) {
Review Comment:
```suggestion
protected ListenableFuture<ResultSet> send(OUT record) {
```
##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java:
##########
@@ -17,130 +17,160 @@
package org.apache.flink.batch.connectors.cassandra;
+import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.io.RichOutputFormat;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.SinkUtils;
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.ResultSet;
-import com.datastax.driver.core.ResultSetFuture;
import com.datastax.driver.core.Session;
-import com.google.common.base.Strings;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicReference;
/**
- * CassandraOutputFormatBase is the common abstract class for writing into
Apache Cassandra.
+ * CassandraOutputFormatBase is the common abstract class for writing into
Apache Cassandra using
+ * output formats.
*
* @param <OUT> Type of the elements to write.
*/
-public abstract class CassandraOutputFormatBase<OUT> extends
RichOutputFormat<OUT> {
+public abstract class CassandraOutputFormatBase<OUT, V> extends
RichOutputFormat<OUT> {
private static final Logger LOG =
LoggerFactory.getLogger(CassandraOutputFormatBase.class);
- private final String insertQuery;
private final ClusterBuilder builder;
+ private Semaphore semaphore;
+ private Duration maxConcurrentRequestsTimeout =
Duration.ofMillis(Long.MAX_VALUE);
+ private int maxConcurrentRequests = Integer.MAX_VALUE;
private transient Cluster cluster;
- private transient Session session;
- private transient PreparedStatement prepared;
- private transient FutureCallback<ResultSet> callback;
- private transient Throwable exception = null;
-
- public CassandraOutputFormatBase(String insertQuery, ClusterBuilder
builder) {
- Preconditions.checkArgument(
- !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or
empty");
+ protected transient Session session;
+ private transient FutureCallback<V> callback;
+ private AtomicReference<Throwable> throwable;
+
+ public CassandraOutputFormatBase(
+ ClusterBuilder builder,
+ int maxConcurrentRequests,
+ Duration maxConcurrentRequestsTimeout) {
Preconditions.checkNotNull(builder, "Builder cannot be null");
-
- this.insertQuery = insertQuery;
this.builder = builder;
+ Preconditions.checkArgument(
+ maxConcurrentRequests > 0, "Max concurrent requests is
expected to be positive");
+ this.maxConcurrentRequests = maxConcurrentRequests;
+ Preconditions.checkNotNull(
+ maxConcurrentRequestsTimeout, "Max concurrent requests timeout
cannot be null");
+ Preconditions.checkArgument(
+ !maxConcurrentRequestsTimeout.isNegative(),
+ "Max concurrent requests timeout is expected to be positive");
+ this.maxConcurrentRequestsTimeout = maxConcurrentRequestsTimeout;
}
+ /** Configure the connection to Cassandra. */
@Override
public void configure(Configuration parameters) {
this.cluster = builder.getCluster();
}
- /**
- * Opens a Session to Cassandra and initializes the prepared statement.
- *
- * @param taskNumber The number of the parallel instance.
- * @throws IOException Thrown, if the output could not be opened due to an
I/O problem.
- */
+ /** Opens a Session to Cassandra . */
@Override
public void open(int taskNumber, int numTasks) throws IOException {
+ throwable = new AtomicReference<>();
+ this.semaphore = new Semaphore(maxConcurrentRequests);
this.session = cluster.connect();
Review Comment:
I'm wondering if it would make sense to move the dependencies on the
datastax driver into the sub-classes (columnar/pojo) at the cost of duplicating
a few lines.
This would allow us to test this class in nicer fashion without mocking.
##########
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBaseTest.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.cassandra.utils.ResultSetFutures;
+import org.apache.flink.core.testutils.CheckedThread;
+import org.apache.flink.streaming.connectors.cassandra.CassandraSinkBase;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+/** Tests for the {@link CassandraSinkBase}. */
+public class CassandraOutputFormatBaseTest {
+
+ private static final long DEFAULT_TEST_TIMEOUT = 5000;
+ private static final Duration DEFAULT_MAX_CONCURRENT_REQUESTS_TIMEOUT =
+ Duration.ofMillis(Long.MAX_VALUE);
+
+ @Test(expected = NoHostAvailableException.class)
Review Comment:
new tests should be written with JUnit5 and assertj.
##########
flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraColumnarOutputFormatBase.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.batch.connectors.cassandra;
+
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.IOException;
+import java.time.Duration;
+
+/**
+ * CassandraColumnarOutputFormatBase is the common abstract class for writing
into Apache Cassandra
+ * using column based output formats.
+ *
+ * @param <OUT> Type of the elements to write.
+ */
+abstract class CassandraColumnarOutputFormatBase<OUT>
+ extends CassandraOutputFormatBase<OUT, ResultSet> {
+ private final String insertQuery;
+ private transient PreparedStatement prepared;
+
+ public CassandraColumnarOutputFormatBase(
+ String insertQuery,
+ ClusterBuilder builder,
+ int maxConcurrentRequests,
+ Duration maxConcurrentRequestsTimeout) {
+ super(builder, maxConcurrentRequests, maxConcurrentRequestsTimeout);
+ Preconditions.checkArgument(
+ !Strings.isNullOrEmpty(insertQuery), "Query cannot be null or
empty");
+ this.insertQuery = insertQuery;
+ }
+
+ /** Opens a Session to Cassandra and initializes the prepared statement. */
Review Comment:
```suggestion
```
The code seems to document this just fine on it's own.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]