flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-9126) Creation of the CassandraPojoInputFormat class to output data into a Custom Cassandra Annotated Pojo
Date Mon, 24 Sep 2018 18:38:01 GMT

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

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

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

 ##########
 File path: flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchPojoExample.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.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 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}-/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();
+			}
+		}));
+
 
 Review comment:
   I think `env.execute("Write");` should be here, not at the end.

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

Mime
View raw message