You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/10/09 08:15:20 UTC

[GitHub] zentol closed pull request #5876: [FLINK-9126] New CassandraPojoInputFormat to output data as a custom annotated Cassandra Pojo

zentol closed pull request #5876: [FLINK-9126] New CassandraPojoInputFormat to output data as a custom annotated Cassandra Pojo
URL: https://github.com/apache/flink/pull/5876
 
 
   

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..60e30669267 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,18 @@
  *
  * @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 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 +64,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..d8212dff314
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormatBase.java
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+/**
+ * InputFormat 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 Logger LOG = LoggerFactory.getLogger(CassandraInputFormatBase.class);
+
+	protected final String query;
+	protected 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.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;
+	}
+
+	@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/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..538b96df681
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraPojoInputFormat.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.core.io.InputSplit;
+import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
+import org.apache.flink.util.Preconditions;
+
+import com.datastax.driver.mapping.Mapper;
+import com.datastax.driver.mapping.MappingManager;
+import com.datastax.driver.mapping.Result;
+
+import java.io.IOException;
+
+/**
+ * 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 Class<OUT> inputClass;
+
+	public CassandraPojoInputFormat(String query, ClusterBuilder builder, Class<OUT> inputClass) {
+		super(query, builder);
+
+		Preconditions.checkArgument(inputClass != null, "InputClass cannot be null");
+
+		this.inputClass = inputClass;
+	}
+
+	@Override
+	public void open(InputSplit split) throws IOException {
+		this.session = cluster.connect();
+		MappingManager manager = new MappingManager(session);
+
+		Mapper<OUT> mapper = manager.mapper(inputClass);
+
+		this.resultSet = mapper.map(session.execute(query));
+	}
+
+	@Override
+	public boolean reachedEnd() throws IOException {
+		return resultSet.isExhausted();
+	}
+
+	@Override
+	public OUT nextRecord(OUT reuse) throws IOException {
+		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..a4f19738e83
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/CustomCassandraAnnotatedPojo.java
@@ -0,0 +1,58 @@
+/*
+ * 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 CassandraInputFormatter.
+ */
+@Table(name = "$TABLE", keyspace = "flink")
+public class CustomCassandraAnnotatedPojo {
+	@Column(name = "id")
+	private String id;
+	@Column(name = "counter")
+	private Integer counter;
+	@Column(name = "batch_id")
+	private Integer 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..7c3c101c4fc
--- /dev/null
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchPojoExample.java
@@ -0,0 +1,80 @@
+/*
+ * 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.Tuple2;
+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 java.util.ArrayList;
+
+/**
+ * This is an example showing the to use the CassandraPojoInput-/CassandraOutputFormats 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 test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’};
+ * CREATE TABLE IF NOT EXISTS test.batches (number int, strings text, PRIMARY KEY(number, strings));
+ */
+public class BatchPojoExample {
+	private static final String INSERT_QUERY = "INSERT INTO test.batches (number, strings) VALUES (?,?);";
+	private static final String SELECT_QUERY = "SELECT number, strings FROM test.batches;";
+
+	/*
+	 *	table script: "CREATE TABLE test.batches (number int, strings text, PRIMARY KEY(number, strings));"
+	 */
+	public static void main(String[] args) throws Exception {
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		ArrayList<Tuple2<Integer, String>> collection = new ArrayList<>(20);
+		for (int i = 0; i < 20; i++) {
+			collection.add(new Tuple2<>(i, "string " + i));
+		}
+
+		DataSet<Tuple2<Integer, String>> dataSet = env.fromCollection(collection);
+
+		dataSet.output(new CassandraTupleOutputFormat<Tuple2<Integer, String>>(INSERT_QUERY, new ClusterBuilder() {
+			@Override
+			protected Cluster buildCluster(Cluster.Builder builder) {
+				return builder.addContactPoints("127.0.0.1").build();
+			}
+		}));
+
+		/*
+		 *	This is for the purpose of showing an example of creating a DataSet using CassandraPojoInputFormat.
+		 */
+		DataSet<CustomCassandraAnnotatedPojo> inputDS = env
+			.createInput(new CassandraPojoInputFormat<>(SELECT_QUERY, new ClusterBuilder() {
+				@Override
+				protected Cluster buildCluster(Cluster.Builder builder) {
+					return builder.addContactPoints("127.0.0.1").build();
+				}
+			}, CustomCassandraAnnotatedPojo.class));
+
+		inputDS.print();
+
+		env.execute("Write");
+	}
+}
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..db2577c9fd7 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;
@@ -470,6 +472,34 @@ public void testCassandraTableSink() throws Exception {
 		Assert.assertTrue("The input data was not completely written to Cassandra", input.isEmpty());
 	}
 
+	@Test
+	public void testCassandraBatchPojoFormat() throws Exception {
+
+		OutputFormat<Tuple3<String, Integer, Integer>> sink = new CassandraTupleOutputFormat<>(injectTableName(INSERT_DATA_QUERY), builder);
+		sink.configure(new Configuration());
+		sink.open(0, 1);
+
+		for (Tuple3<String, Integer, Integer> value : collection) {
+			sink.writeRecord(value);
+		}
+
+		sink.close();
+
+		InputFormat<CustomCassandraAnnotatedPojo, InputSplit> source = new CassandraPojoInputFormat<>(injectTableName(SELECT_DATA_QUERY), builder, CustomCassandraAnnotatedPojo.class);
+		source.configure(new Configuration());
+		source.open(null);
+
+		List<CustomCassandraAnnotatedPojo> result = new ArrayList<>();
+
+		while (!source.reachedEnd()) {
+			CustomCassandraAnnotatedPojo temp = source.nextRecord(new CustomCassandraAnnotatedPojo());
+			result.add(temp);
+		}
+
+		source.close();
+		Assert.assertEquals(20, result.size());
+	}
+
 	@Test
 	public void testCassandraBatchTupleFormat() throws Exception {
 		OutputFormat<Tuple3<String, Integer, Integer>> sink = new CassandraOutputFormat<>(injectTableName(INSERT_DATA_QUERY), builder);


 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services