[ https://issues.apache.org/jira/browse/FLINK-9126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16642942#comment-16642942 ]
ASF GitHub Bot commented on FLINK-9126: --------------------------------------- zentol closed pull request #6735: [FLINK-9126] New CassandraPojoInputFormat to output data as a custom annotated Cassandra Pojo URL: https://github.com/apache/flink/pull/6735 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java index e0806fe2056..cc07fb70799 100644 --- a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java +++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java @@ -17,25 +17,12 @@ package org.apache.flink.batch.connectors.cassandra; -import org.apache.flink.api.common.io.DefaultInputSplitAssigner; -import org.apache.flink.api.common.io.NonParallelInput; -import org.apache.flink.api.common.io.RichInputFormat; -import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; -import org.apache.flink.core.io.InputSplitAssigner; 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.ResultSet; import com.datastax.driver.core.Row; -import com.datastax.driver.core.Session; -import com.google.common.base.Strings; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; @@ -44,38 +31,19 @@ * * @param <OUT> type of Tuple */ -public class CassandraInputFormat<OUT extends Tuple> extends RichInputFormat<OUT, InputSplit> implements NonParallelInput { - private static final Logger LOG = LoggerFactory.getLogger(CassandraInputFormat.class); +public class CassandraInputFormat<OUT extends Tuple> extends CassandraInputFormatBase<OUT> { - private final String query; - private final ClusterBuilder builder; - - private transient Cluster cluster; - private transient Session session; + private static final long serialVersionUID = 3642323148032444264L; private transient ResultSet resultSet; public CassandraInputFormat(String query, ClusterBuilder builder) { - Preconditions.checkArgument(!Strings.isNullOrEmpty(query), "Query cannot be null or empty"); - Preconditions.checkArgument(builder != null, "Builder cannot be null"); - - this.query = query; - this.builder = builder; - } - - @Override - public void configure(Configuration parameters) { - this.cluster = builder.getCluster(); - } - - @Override - public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { - return cachedStatistics; + super(query, builder); } /** * Opens a Session and executes the query. * - * @param ignored + * @param ignored because parameter is not parallelizable. * @throws IOException */ @Override @@ -97,37 +65,4 @@ public OUT nextRecord(OUT reuse) throws IOException { } return reuse; } - - @Override - public InputSplit[] createInputSplits(int minNumSplits) throws IOException { - GenericInputSplit[] split = {new GenericInputSplit(0, 1)}; - return split; - } - - @Override - public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) { - return new DefaultInputSplitAssigner(inputSplits); - } - - /** - * Closes all resources used. - */ - @Override - public void close() throws IOException { - try { - if (session != null) { - session.close(); - } - } catch (Exception e) { - LOG.error("Error while closing session.", e); - } - - try { - if (cluster != null) { - cluster.close(); - } - } catch (Exception e) { - LOG.error("Error while closing cluster.", e); - } - } } diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormatBase.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormatBase.java new file mode 100644 index 00000000000..9c6978f7c37 --- /dev/null +++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormatBase.java @@ -0,0 +1,103 @@ +/* + * 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.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.NonParallelInput; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.GenericInputSplit; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava18.com.google.common.base.Strings; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Session; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Base class for {@link RichInputFormat} to read data from Apache Cassandra and generate a custom Cassandra annotated object. + * + * @param <OUT> type of inputClass + */ +public abstract class CassandraInputFormatBase<OUT> extends RichInputFormat<OUT, InputSplit> implements NonParallelInput { + private static final long serialVersionUID = -1519372881115104601L; + protected final Logger logger = LoggerFactory.getLogger(getClass()); + + protected final String query; + private final ClusterBuilder builder; + + protected transient Cluster cluster; + protected transient Session session; + + public CassandraInputFormatBase(String query, ClusterBuilder builder){ + Preconditions.checkArgument(!Strings.isNullOrEmpty(query), "Query cannot be null or empty"); + Preconditions.checkNotNull(builder, "Builder cannot be null"); + + this.query = query; + this.builder = builder; + } + + @Override + public void configure(Configuration parameters) { + this.cluster = builder.getCluster(); + } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { + return cachedStatistics; + } + + @Override + public InputSplit[] createInputSplits(int minNumSplits) { + return new GenericInputSplit[]{new GenericInputSplit(0, 1)}; + } + + @Override + public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) { + return new DefaultInputSplitAssigner(inputSplits); + } + + /** + * Closes all resources used. + */ + @Override + public void close() { + try { + if (session != null) { + session.close(); + } + } catch (Exception e) { + logger.error("Error while closing session.", e); + } + + try { + if (cluster != null) { + cluster.close(); + } + } catch (Exception e) { + logger.error("Error while closing cluster.", e); + } + } + +} diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java index e458a2803d6..24f11575b7b 100644 --- a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java +++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraOutputFormatBase.java @@ -54,7 +54,7 @@ public CassandraOutputFormatBase(String insertQuery, ClusterBuilder builder) { Preconditions.checkArgument(!Strings.isNullOrEmpty(insertQuery), "Query cannot be null or empty"); - Preconditions.checkArgument(builder != null, "Builder cannot be null"); + Preconditions.checkNotNull(builder, "Builder cannot be null"); this.insertQuery = insertQuery; this.builder = builder; diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraPojoInputFormat.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraPojoInputFormat.java new file mode 100644 index 00000000000..db05fb67760 --- /dev/null +++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraPojoInputFormat.java @@ -0,0 +1,77 @@ +/* + * 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.core.io.InputSplit; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; +import org.apache.flink.streaming.connectors.cassandra.MapperOptions; +import org.apache.flink.util.Preconditions; + +import com.datastax.driver.mapping.Mapper; +import com.datastax.driver.mapping.MappingManager; +import com.datastax.driver.mapping.Result; + +/** + * InputFormat to read data from Apache Cassandra and generate a custom Cassandra annotated object. + * + * @param <OUT> type of inputClass + */ +public class CassandraPojoInputFormat<OUT> extends CassandraInputFormatBase<OUT> { + + private static final long serialVersionUID = 1992091320180905115L; + + private transient Result<OUT> resultSet; + private final MapperOptions mapperOptions; + private final Class<OUT> inputClass; + + public CassandraPojoInputFormat(String query, ClusterBuilder builder, Class<OUT> inputClass) { + this(query, builder, inputClass, null); + } + + public CassandraPojoInputFormat(String query, ClusterBuilder builder, Class<OUT> inputClass, MapperOptions mapperOptions) { + super(query, builder); + this.mapperOptions = mapperOptions; + this.inputClass = Preconditions.checkNotNull(inputClass, "InputClass cannot be null"); + } + + @Override + public void open(InputSplit split) { + this.session = cluster.connect(); + MappingManager manager = new MappingManager(session); + + Mapper<OUT> mapper = manager.mapper(inputClass); + + if (mapperOptions != null) { + Mapper.Option[] optionsArray = mapperOptions.getMapperOptions(); + if (optionsArray != null) { + mapper.setDefaultGetOptions(optionsArray); + } + } + this.resultSet = mapper.map(session.execute(query)); + } + + @Override + public boolean reachedEnd() { + return resultSet.isExhausted(); + } + + @Override + public OUT nextRecord(OUT reuse) { + return resultSet.one(); + } +} diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CustomCassandraAnnotatedPojo.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CustomCassandraAnnotatedPojo.java new file mode 100644 index 00000000000..14dc69c84c3 --- /dev/null +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CustomCassandraAnnotatedPojo.java @@ -0,0 +1,72 @@ +/* + * 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 com.datastax.driver.mapping.annotations.Column; +import com.datastax.driver.mapping.annotations.Table; + +/** + * Example of Cassandra Annotated POJO class for use with {@link CassandraPojoInputFormat}. + */ +@Table(name = CustomCassandraAnnotatedPojo.TABLE_NAME, keyspace = "flink") +public class CustomCassandraAnnotatedPojo { + + public static final String TABLE_NAME = "batches"; + + @Column(name = "id") + private String id; + @Column(name = "counter") + private Integer counter; + @Column(name = "batch_id") + private Integer batchId; + + /** + * Necessary for the driver's mapper instanciation. + */ + public CustomCassandraAnnotatedPojo(){} + + public CustomCassandraAnnotatedPojo(String id, Integer counter, Integer batchId) { + this.id = id; + this.counter = counter; + this.batchId = batchId; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public Integer getCounter() { + return counter; + } + + public void setCounter(Integer counter) { + this.counter = counter; + } + + public Integer getBatchId() { + return batchId; + } + + public void setBatchId(Integer batchId) { + this.batchId = batchId; + } +} diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchPojoExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchPojoExample.java new file mode 100644 index 00000000000..8b4e4b125cd --- /dev/null +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchPojoExample.java @@ -0,0 +1,83 @@ +/* + * 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.example; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.batch.connectors.cassandra.CassandraPojoInputFormat; +import org.apache.flink.batch.connectors.cassandra.CassandraTupleOutputFormat; +import org.apache.flink.batch.connectors.cassandra.CustomCassandraAnnotatedPojo; +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.mapping.Mapper; + +import java.util.ArrayList; + +/** + * This is an example showing the to use the {@link CassandraPojoInputFormat}/{@link CassandraTupleOutputFormat} in the Batch API. + * + * <p>The example assumes that a table exists in a local cassandra database, according to the following queries: + * CREATE KEYSPACE IF NOT EXISTS flink WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’}; + * CREATE TABLE IF NOT EXISTS flink.batches (id text, counter int, batch_id int, PRIMARY KEY(id, counter, batchId)); + */ +public class BatchPojoExample { + private static final String INSERT_QUERY = "INSERT INTO flink.batches (id, counter, batch_id) VALUES (?,?,?);"; + private static final String SELECT_QUERY = "SELECT id, counter, batch_id FROM flink.batches;"; + + public static void main(String[] args) throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + ArrayList<Tuple3<String, Integer, Integer>> collection = new ArrayList<>(20); + for (int i = 0; i < 20; i++) { + collection.add(new Tuple3<>("string " + i, i, i)); + } + + DataSet<Tuple3<String, Integer, Integer>> dataSet = env.fromCollection(collection); + + ClusterBuilder clusterBuilder = new ClusterBuilder() { + private static final long serialVersionUID = -1754532803757154795L; + + @Override + protected Cluster buildCluster(Cluster.Builder builder) { + return builder.addContactPoints("127.0.0.1").build(); + } + }; + + dataSet.output(new CassandraTupleOutputFormat<>(INSERT_QUERY, clusterBuilder)); + + env.execute("Write"); + + /* + * This is for the purpose of showing an example of creating a DataSet using CassandraPojoInputFormat. + */ + DataSet<CustomCassandraAnnotatedPojo> inputDS = env + .createInput(new CassandraPojoInputFormat<>( + SELECT_QUERY, + clusterBuilder, + CustomCassandraAnnotatedPojo.class, + () -> new Mapper.Option[]{Mapper.Option.consistencyLevel(ConsistencyLevel.ANY)} + )); + + inputDS.print(); + } +} diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java index a2a77777ebf..a3b7093dd25 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java @@ -31,8 +31,10 @@ import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo; import org.apache.flink.batch.connectors.cassandra.CassandraInputFormat; import org.apache.flink.batch.connectors.cassandra.CassandraOutputFormat; +import org.apache.flink.batch.connectors.cassandra.CassandraPojoInputFormat; import org.apache.flink.batch.connectors.cassandra.CassandraRowOutputFormat; import org.apache.flink.batch.connectors.cassandra.CassandraTupleOutputFormat; +import org.apache.flink.batch.connectors.cassandra.CustomCassandraAnnotatedPojo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.datastream.DataStream; @@ -66,14 +68,19 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Random; import java.util.Scanner; import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import scala.collection.JavaConverters; import scala.collection.Seq; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.samePropertyValuesAs; import static org.junit.Assert.assertTrue; /** @@ -375,15 +382,15 @@ public void testCassandraCommitter() throws Exception { @Test public void testCassandraTupleAtLeastOnceSink() throws Exception { CassandraTupleSink<Tuple3<String, Integer, Integer>> sink = new CassandraTupleSink<>(injectTableName(INSERT_DATA_QUERY), builder); - - sink.open(new Configuration()); - - for (Tuple3<String, Integer, Integer> value : collection) { - sink.send(value); + try { + sink.open(new Configuration()); + for (Tuple3<String, Integer, Integer> value : collection) { + sink.send(value); + } + } finally { + sink.close(); } - sink.close(); - ResultSet rs = session.execute(injectTableName(SELECT_DATA_QUERY)); Assert.assertEquals(20, rs.all().size()); } @@ -391,15 +398,15 @@ public void testCassandraTupleAtLeastOnceSink() throws Exception { @Test public void testCassandraRowAtLeastOnceSink() throws Exception { CassandraRowSink sink = new CassandraRowSink(FIELD_TYPES.length, injectTableName(INSERT_DATA_QUERY), builder); - - sink.open(new Configuration()); - - for (Row value : rowCollection) { - sink.send(value); + try { + sink.open(new Configuration()); + for (Row value : rowCollection) { + sink.send(value); + } + } finally { + sink.close(); } - sink.close(); - ResultSet rs = session.execute(injectTableName(SELECT_DATA_QUERY)); Assert.assertEquals(20, rs.all().size()); } @@ -409,15 +416,15 @@ public void testCassandraPojoAtLeastOnceSink() throws Exception { session.execute(CREATE_TABLE_QUERY.replace(TABLE_NAME_VARIABLE, "test")); CassandraPojoSink<Pojo> sink = new CassandraPojoSink<>(Pojo.class, builder); - - sink.open(new Configuration()); - - for (int x = 0; x < 20; x++) { - sink.send(new Pojo(UUID.randomUUID().toString(), x, 0)); + try { + sink.open(new Configuration()); + for (int x = 0; x < 20; x++) { + sink.send(new Pojo(UUID.randomUUID().toString(), x, 0)); + } + } finally { + sink.close(); } - sink.close(); - ResultSet rs = session.execute(SELECT_DATA_QUERY.replace(TABLE_NAME_VARIABLE, "test")); Assert.assertEquals(20, rs.all().size()); } @@ -427,16 +434,15 @@ public void testCassandraPojoNoAnnotatedKeyspaceAtLeastOnceSink() throws Excepti session.execute(CREATE_TABLE_QUERY.replace(TABLE_NAME_VARIABLE, "testPojoNoAnnotatedKeyspace")); CassandraPojoSink<PojoNoAnnotatedKeyspace> sink = new CassandraPojoSink<>(PojoNoAnnotatedKeyspace.class, builder, "flink"); + try { + sink.open(new Configuration()); + for (int x = 0; x < 20; x++) { + sink.send(new PojoNoAnnotatedKeyspace(UUID.randomUUID().toString(), x, 0)); + } - Configuration configuration = new Configuration(); - sink.open(configuration); - - for (int x = 0; x < 20; x++) { - sink.send(new PojoNoAnnotatedKeyspace(UUID.randomUUID().toString(), x, 0)); + } finally { + sink.close(); } - - sink.close(); - ResultSet rs = session.execute(SELECT_DATA_QUERY.replace(TABLE_NAME_VARIABLE, "testPojoNoAnnotatedKeyspace")); Assert.assertEquals(20, rs.all().size()); } @@ -471,53 +477,97 @@ public void testCassandraTableSink() throws Exception { } @Test - public void testCassandraBatchTupleFormat() throws Exception { - OutputFormat<Tuple3<String, Integer, Integer>> sink = new CassandraOutputFormat<>(injectTableName(INSERT_DATA_QUERY), builder); - sink.configure(new Configuration()); - sink.open(0, 1); + public void testCassandraBatchPojoFormat() throws Exception { + + session.execute(CREATE_TABLE_QUERY.replace(TABLE_NAME_VARIABLE, CustomCassandraAnnotatedPojo.TABLE_NAME)); + + CassandraPojoSink<CustomCassandraAnnotatedPojo> sink = new CassandraPojoSink<>(CustomCassandraAnnotatedPojo.class, builder); + List<CustomCassandraAnnotatedPojo> customCassandraAnnotatedPojos = IntStream.range(0, 20) + .mapToObj(x -> new CustomCassandraAnnotatedPojo(UUID.randomUUID().toString(), x, 0)) + .collect(Collectors.toList()); + try { + sink.open(new Configuration()); + customCassandraAnnotatedPojos.forEach(sink::send); + } finally { + sink.close(); + } + ResultSet rs = session.execute(SELECT_DATA_QUERY.replace(TABLE_NAME_VARIABLE, CustomCassandraAnnotatedPojo.TABLE_NAME)); + Assert.assertEquals(20, rs.all().size()); - for (Tuple3<String, Integer, Integer> value : collection) { - sink.writeRecord(value); + InputFormat<CustomCassandraAnnotatedPojo, InputSplit> source = new CassandraPojoInputFormat<>(SELECT_DATA_QUERY.replace(TABLE_NAME_VARIABLE, "batches"), builder, CustomCassandraAnnotatedPojo.class); + List<CustomCassandraAnnotatedPojo> result = new ArrayList<>(); + + try { + source.configure(new Configuration()); + source.open(null); + while (!source.reachedEnd()) { + CustomCassandraAnnotatedPojo temp = source.nextRecord(null); + result.add(temp); + } + } finally { + source.close(); } - sink.close(); + Assert.assertEquals(20, result.size()); + result.sort(Comparator.comparingInt(CustomCassandraAnnotatedPojo::getCounter)); + customCassandraAnnotatedPojos.sort(Comparator.comparingInt(CustomCassandraAnnotatedPojo::getCounter)); - sink = new CassandraTupleOutputFormat<>(injectTableName(INSERT_DATA_QUERY), builder); - sink.configure(new Configuration()); - sink.open(0, 1); + assertThat(result, samePropertyValuesAs(customCassandraAnnotatedPojos)); + } - for (Tuple3<String, Integer, Integer> value : collection) { - sink.writeRecord(value); + @Test + public void testCassandraBatchTupleFormat() throws Exception { + OutputFormat<Tuple3<String, Integer, Integer>> sink = new CassandraOutputFormat<>(injectTableName(INSERT_DATA_QUERY), builder); + try { + sink.configure(new Configuration()); + sink.open(0, 1); + for (Tuple3<String, Integer, Integer> value : collection) { + sink.writeRecord(value); + } + } finally { + sink.close(); } - sink.close(); + sink = new CassandraTupleOutputFormat<>(injectTableName(INSERT_DATA_QUERY), builder); + try { + sink.configure(new Configuration()); + sink.open(0, 1); + for (Tuple3<String, Integer, Integer> value : collection) { + sink.writeRecord(value); + } + } finally { + sink.close(); + } InputFormat<Tuple3<String, Integer, Integer>, InputSplit> source = new CassandraInputFormat<>(injectTableName(SELECT_DATA_QUERY), builder); - source.configure(new Configuration()); - source.open(null); - List<Tuple3<String, Integer, Integer>> result = new ArrayList<>(); - - while (!source.reachedEnd()) { - result.add(source.nextRecord(new Tuple3<String, Integer, Integer>())); + try { + source.configure(new Configuration()); + source.open(null); + while (!source.reachedEnd()) { + result.add(source.nextRecord(new Tuple3<String, Integer, Integer>())); + } + } finally { + source.close(); } - source.close(); Assert.assertEquals(20, result.size()); } @Test public void testCassandraBatchRowFormat() throws Exception { OutputFormat<Row> sink = new CassandraRowOutputFormat(injectTableName(INSERT_DATA_QUERY), builder); - sink.configure(new Configuration()); - sink.open(0, 1); + try { + sink.configure(new Configuration()); + sink.open(0, 1); + for (Row value : rowCollection) { + sink.writeRecord(value); + } + } finally { - for (Row value : rowCollection) { - sink.writeRecord(value); + sink.close(); } - sink.close(); - ResultSet rs = session.execute(injectTableName(SELECT_DATA_QUERY)); List<com.datastax.driver.core.Row> rows = rs.all(); Assert.assertEquals(rowCollection.size(), rows.size()); @@ -557,12 +607,14 @@ public void testCassandraScalaTupleAtLeastSink() throws Exception { for (int i = 0; i < 20; i++) { scalaTupleCollection.add(new scala.Tuple3<>(UUID.randomUUID().toString(), i, 0)); } - - sink.open(new Configuration()); - for (scala.Tuple3<String, Integer, Integer> value : scalaTupleCollection) { - sink.invoke(value, SinkContextUtil.forTimestamp(0)); + try { + sink.open(new Configuration()); + for (scala.Tuple3<String, Integer, Integer> value : scalaTupleCollection) { + sink.invoke(value, SinkContextUtil.forTimestamp(0)); + } + } finally { + sink.close(); } - sink.close(); ResultSet rs = session.execute(injectTableName(SELECT_DATA_QUERY)); List<com.datastax.driver.core.Row> rows = rs.all(); ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Creation of the CassandraPojoInputFormat class to output data into a Custom > Cassandra Annotated Pojo > ---------------------------------------------------------------------------------------------------- > > Key: FLINK-9126 > URL: https://issues.apache.org/jira/browse/FLINK-9126 > Project: Flink > Issue Type: New Feature > Components: DataSet API > Affects Versions: 1.5.0, 1.4.2 > Reporter: Jeffrey Carter > Assignee: Jeffrey Carter > Priority: Minor > Labels: InputFormat, cassandra, features, pull-request-available > Fix For: 1.7.0 > > Attachments: CassandraPojoInputFormatText.rtf > > Original Estimate: 24h > Remaining Estimate: 24h > > Currently the DataSet API only has the ability to output data received from > Cassandra as a source in as a Tuple. This would be allow the data to be > output as a custom POJO that the user has created that has been annotated > using Datastax API. This would remove the need of very long Tuples to be > created by the DataSet and then mapped to the custom POJO. > > -The changes to the CassandraInputFormat object would be minimal, but would > require importing the Datastax API into the class-. Another option is to make > a similar, but slightly different class called CassandraPojoInputFormat. > I have already gotten code for this working in my own project, but want other > thoughts as to the best way this should go about being implemented. > > //Example of its use in main > CassandraPojoInputFormat<CustomCassandraPojo> cassandraInputFormat = new > CassandraPojoInputFormat<>(queryToRun, defaultClusterBuilder, > CustomCassandraPojo.class); > cassandraInputFormat.configure(null); > cassandraInputFormat.open(null); > DataSet<CustomCassandraPojo> outputTestSet = > exEnv.createInput(cassandraInputFormat, TypeInformation.of(new > TypeHint<CustomCassandraPojo>(){})); > > //The class that I currently have set up > [^CassandraPojoInputFormatText.rtf] > > Will make another Jira Issue for the Output version next if this is approved -- This message was sent by Atlassian JIRA (v7.6.3#76005)