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

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

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

 ##########
 File path: 
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");
 
 Review comment:
   it will throw a NullPointerException instead of IllegallArgumentException, 
like others classes of the connector, i think we should keep checkArgument

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

Reply via email to