You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/10/01 11:45:01 UTC

[jira] [Commented] (FLINK-9126) Creation of the CassandraPojoInputFormat class to output data into a Custom Cassandra Annotated Pojo

    [ https://issues.apache.org/jira/browse/FLINK-9126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16633883#comment-16633883 ] 

ASF GitHub Bot commented on FLINK-9126:
---------------------------------------

zentol commented on a change in pull request #6735: [FLINK-9126] New CassandraPojoInputFormat to output data as a custom annotated Cassandra Pojo
URL: https://github.com/apache/flink/pull/6735#discussion_r221575749
 
 

 ##########
 File path: flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchPojoExample.java
 ##########
 @@ -0,0 +1,78 @@
+/*
+ * 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)}));
 
 Review comment:
   place arguments on separate lines for readability

----------------------------------------------------------------
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


> 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)