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, 01 Oct 2018 11:45:01 GMT

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

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_r221578896
 
 

 ##########
 File path: flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
 ##########
 @@ -470,6 +477,41 @@ public void testCassandraTableSink() throws Exception {
 		Assert.assertTrue("The input data was not completely written to Cassandra", input.isEmpty());
 	}
 
+	@Test
+	public void testCassandraBatchPojoFormat() throws Exception {
+
+		session.execute(CREATE_TABLE_QUERY.replace(TABLE_NAME_VARIABLE, "batches"));
+
+		CassandraPojoSink<CustomCassandraAnnotatedPojo> sink = new CassandraPojoSink<>(CustomCassandraAnnotatedPojo.class,
builder);
+		sink.open(new Configuration());
+
+		List<CustomCassandraAnnotatedPojo> customCassandraAnnotatedPojos = IntStream.range(0,
20)
+			.mapToObj(x -> new CustomCassandraAnnotatedPojo(UUID.randomUUID().toString(), x, 0))
+			.collect(Collectors.toList());
+
+		customCassandraAnnotatedPojos.forEach(sink::send);
+		sink.close();
+
+		InputFormat<CustomCassandraAnnotatedPojo, InputSplit> source = new CassandraPojoInputFormat<>(SELECT_DATA_QUERY.replace(TABLE_NAME_VARIABLE,
"batches"), builder, CustomCassandraAnnotatedPojo.class);
+		source.configure(new Configuration());
+		source.open(null);
+
+		List<CustomCassandraAnnotatedPojo> result = new ArrayList<>();
+		while (!source.reachedEnd()) {
+			CustomCassandraAnnotatedPojo temp = source.nextRecord(null);
+			result.add(temp);
+		}
+
+		source.close();
+		Assert.assertEquals(20, result.size());
+		result.sort(Comparator.comparingInt(CustomCassandraAnnotatedPojo::getCounter));
+		customCassandraAnnotatedPojos.sort(Comparator.comparingInt(CustomCassandraAnnotatedPojo::getCounter));
+
+		for (int i = 0; i < result.size(); i++) {
+			assertThat(result.get(i), samePropertyValuesAs(customCassandraAnnotatedPojos.get(i)));
 
 Review comment:
   since they are sorted already I believe we could just compare the lists directly.

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