[
https://issues.apache.org/jira/browse/FLINK-5101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15834416#comment-15834416
]
ASF GitHub Bot commented on FLINK-5101:
---------------------------------------
Github user StephanEwen commented on a diff in the pull request:
https://github.com/apache/flink/pull/2866#discussion_r97310987
--- Diff:
flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
---
@@ -371,70 +351,77 @@ public void testCassandraCommitter() throws Exception
{
@Test
public void testCassandraTupleAtLeastOnceSink() throws Exception {
- StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
+ CassandraTupleSink<Tuple3<String, Integer, Integer>> sink = new
CassandraTupleSink<>(injectTableName(INSERT_DATA_QUERY), builder);
+
+ sink.open(new Configuration());
- DataStream<Tuple3<String, Integer, Integer>> source =
env.fromCollection(collection);
- source.addSink(new CassandraTupleSink<Tuple3<String, Integer,
Integer>>(INSERT_DATA_QUERY, builder));
+ for (Tuple3<String, Integer, Integer> value : collection) {
+ sink.send(value);
+ }
- env.execute();
+ sink.close();
- ResultSet rs = session.execute(SELECT_DATA_QUERY);
+ synchronized (sink.updatesPending) {
+ if (sink.updatesPending.get() != 0) {
+ sink.updatesPending.wait();
+ }
+ }
+
+ ResultSet rs =
session.execute(injectTableName(SELECT_DATA_QUERY));
Assert.assertEquals(20, rs.all().size());
}
@Test
public void testCassandraPojoAtLeastOnceSink() throws Exception {
- StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
-
- DataStreamSource<Pojo> source = env
- .addSource(new SourceFunction<Pojo>() {
-
- private boolean running = true;
- private volatile int cnt = 0;
-
- @Override
- public void run(SourceContext<Pojo> ctx) throws
Exception {
- while (running) {
- ctx.collect(new
Pojo(UUID.randomUUID().toString(), cnt, 0));
- cnt++;
- if (cnt == 20) {
- cancel();
- }
- }
- }
+ session.execute(CREATE_TABLE_QUERY.replace(TABLE_NAME_VARIABLE,
"test"));
- @Override
- public void cancel() {
- running = false;
- }
- });
+ CassandraPojoSink<Pojo> sink = new
CassandraPojoSink<>(Pojo.class, builder);
- source.addSink(new CassandraPojoSink<>(Pojo.class, builder));
+ sink.open(new Configuration());
- env.execute();
+ for (int x = 0; x < 20; x++) {
+ sink.send(new Pojo(UUID.randomUUID().toString(), x, 0));
+ }
+
+ sink.close();
+
+ synchronized (sink.updatesPending) {
--- End diff --
Same as above
> Test CassandraConnectorITCase instable
> --------------------------------------
>
> Key: FLINK-5101
> URL: https://issues.apache.org/jira/browse/FLINK-5101
> Project: Flink
> Issue Type: Bug
> Components: Cassandra Connector
> Reporter: Stefan Richter
> Assignee: Chesnay Schepler
>
> I observed this test fail on Travis (very rarely):
>
> Running
> org.apache.flink.streaming.connectors.cassandra.CassandraConnectorITCase
> Tests run: 7, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 80.843 sec
> <<< FAILURE! - in
> org.apache.flink.streaming.connectors.cassandra.CassandraConnectorITCase
> testCassandraBatchFormats(org.apache.flink.streaming.connectors.cassandra.CassandraConnectorITCase)
> Time elapsed: 5.82 sec <<< FAILURE!
> java.lang.AssertionError: expected:<40> but was:<20>
> at org.junit.Assert.fail(Assert.java:88)
> at org.junit.Assert.failNotEquals(Assert.java:834)
> at org.junit.Assert.assertEquals(Assert.java:645)
> at org.junit.Assert.assertEquals(Assert.java:631)
> at
> org.apache.flink.streaming.connectors.cassandra.CassandraConnectorITCase.testCassandraBatchFormats(CassandraConnectorITCase.java:442)
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)