Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/11863#discussion_r69082952
--- Diff:
external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java
---
@@ -0,0 +1,84 @@
+/*
+ * 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.spark.streaming.kafka010;
+
+import java.io.Serializable;
+import java.util.*;
+
+import scala.collection.JavaConverters;
+
+import org.apache.kafka.common.TopicPartition;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class JavaConsumerStrategySuite implements Serializable {
+
+ @Test
+ public void testConsumerStrategyConstructors() {
+ final String topic1 = "topic1";
+ final Collection<String> topics = Arrays.asList(topic1);
+ final scala.collection.Iterable<String> sTopics =
+ JavaConverters.collectionAsScalaIterableConverter(topics).asScala();
+ final TopicPartition tp1 = new TopicPartition(topic1, 0);
+ final TopicPartition tp2 = new TopicPartition(topic1, 1);
+ final Collection<TopicPartition> parts = Arrays.asList(tp1, tp2);
+ final scala.collection.Iterable<TopicPartition> sParts =
+ JavaConverters.collectionAsScalaIterableConverter(parts).asScala();
+ final Map<String, Object> kafkaParams = new HashMap<String, Object>();
+ kafkaParams.put("bootstrap.servers", "not used");
+ final scala.collection.Map<String, Object> sKafkaParams =
+ JavaConverters.mapAsScalaMapConverter(kafkaParams).asScala();
+ final Map<TopicPartition, Object> offsets = new HashMap<>();
+ offsets.put(tp1, 23L);
+ final scala.collection.Map<TopicPartition, Object> sOffsets =
+ JavaConverters.mapAsScalaMapConverter(offsets).asScala();
+
+ // make sure constructors can be called from java
+ final ConsumerStrategy<String, String> sub0 =
+ Subscribe.<String, String>apply(topics, kafkaParams, offsets);
--- End diff --
This is seems to break in scala 2.10 and not scala 2.11. This is very weird.
Merging this PR broke 2.10 builds -
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-sbt-scala-2.10/1947/console
```
[error]
/home/jenkins/workspace/spark-master-compile-sbt-scala-2.10/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java:54:
error: incompatible types: Collection<String> cannot be converted to
Iterable<String>
[error] Subscribe.<String, String>apply(topics, kafkaParams, offsets);
[error] ^
[error]
/home/jenkins/workspace/spark-master-compile-sbt-scala-2.10/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java:69:
error: incompatible types: Collection<TopicPartition> cannot be converted to
Iterable<TopicPartition>
[error] Assign.<String, String>apply(parts, kafkaParams, offsets);
[error] ^
```
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]