yunfengzhou-hub commented on code in PR #160: URL: https://github.com/apache/flink-ml/pull/160#discussion_r1003935113
########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/randomsplitter/RandomSplitterParams.java: ########## @@ -0,0 +1,69 @@ +/* + * 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.ml.feature.randomsplitter; + +import org.apache.flink.ml.param.DoubleArrayParam; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.param.ParamValidator; +import org.apache.flink.ml.param.WithParams; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +/** + * Params of {@link RandomSplitter}. + * + * @param <T> The class type of this instance. + */ +public interface RandomSplitterParams<T> extends WithParams<T> { + Param<Double[]> FRACTIONS = + new DoubleArrayParam( + "fractions", Review Comment: Do you think it would be more intuitive to use `weights` instead of `fractions`? `fractions` seems to be thresholds of the randomly generated doubles in the implementation detail of this algorithm, and I think `weights` might be simpler and easier to understand than `fractions`. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/randomsplitter/RandomSplitterParams.java: ########## @@ -0,0 +1,69 @@ +/* + * 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.ml.feature.randomsplitter; + +import org.apache.flink.ml.param.DoubleArrayParam; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.param.ParamValidator; +import org.apache.flink.ml.param.WithParams; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +/** + * Params of {@link RandomSplitter}. + * + * @param <T> The class type of this instance. + */ +public interface RandomSplitterParams<T> extends WithParams<T> { + Param<Double[]> FRACTIONS = + new DoubleArrayParam( + "fractions", + "The fractions of data splitting.", + new Double[] {0.5}, Review Comment: The `weights` parameter of Spark's `RDD.randomSplit()` is a required parameter, thus it might be better to make the default value of our `fraction` to be `null` to match Spark's behavior. ########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/RandomSplitterTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.ml.feature; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.feature.randomsplitter.RandomSplitter; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** Tests {@link RandomSplitter}. */ +public class RandomSplitterTest extends AbstractTestBase { + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(4); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + + tEnv = StreamTableEnvironment.create(env); + } + + private Table getTable(int size) { + List<Row> data = new ArrayList<>(); + for (int i = 0; i < size; ++i) { + data.add(Row.of(i)); + } + return tEnv.fromDataStream(env.fromCollection(data)); + } + + @Test + public void testParam() { + RandomSplitter splitter = new RandomSplitter(); + assertEquals(0.5, splitter.getFractions()[0], 1.0e-5); + + splitter.setFractions(0.3, 0.4); + assertArrayEquals(new Double[] {0.3, 0.4}, splitter.getFractions()); + } + + @Test + public void testOutputSchema() { + Table tempTable = + tEnv.fromDataStream(env.fromElements(Row.of("", ""))) + .as("test_input", "dummy_input"); + + RandomSplitter splitter = new RandomSplitter(); + Table output = splitter.transform(tempTable)[0]; Review Comment: Let's check the length of output and the schema of each of the multiple output tables. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/randomsplitter/RandomSplitter.java: ########## @@ -0,0 +1,135 @@ +/* + * 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.ml.feature.randomsplitter; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.ml.api.AlgoOperator; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableImpl; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Random; + +/** + * An AlgoOperator which splits a datastream into N datastreams according to the given fractions. + */ +public class RandomSplitter + implements AlgoOperator<RandomSplitter>, RandomSplitterParams<RandomSplitter> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + + public RandomSplitter() { + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + } + + @Override + public Table[] transform(Table... inputs) { + Preconditions.checkArgument(inputs.length == 1); + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment(); + DataStream<Tuple2<Integer, Row>> rowWithFlag; + final Double[] fractions = getFractions(); + rowWithFlag = + tEnv.toDataStream(inputs[0]) + .map( + new MapFunction<Row, Tuple2<Integer, Row>>() { Review Comment: It might be better to use DataStream's side output mechanism to split a stream into multiple ones. Compared with side outputs, the current implementation uses more network resources and thus might not be good enough. You may refer to AgglomerativeClustering for how side output has been used. ########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/RandomSplitterTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.ml.feature; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.feature.randomsplitter.RandomSplitter; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** Tests {@link RandomSplitter}. */ +public class RandomSplitterTest extends AbstractTestBase { + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(4); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + + tEnv = StreamTableEnvironment.create(env); + } + + private Table getTable(int size) { + List<Row> data = new ArrayList<>(); + for (int i = 0; i < size; ++i) { + data.add(Row.of(i)); + } + return tEnv.fromDataStream(env.fromCollection(data)); + } + + @Test + public void testParam() { + RandomSplitter splitter = new RandomSplitter(); + assertEquals(0.5, splitter.getFractions()[0], 1.0e-5); + + splitter.setFractions(0.3, 0.4); + assertArrayEquals(new Double[] {0.3, 0.4}, splitter.getFractions()); + } + + @Test + public void testOutputSchema() { + Table tempTable = + tEnv.fromDataStream(env.fromElements(Row.of("", ""))) + .as("test_input", "dummy_input"); + + RandomSplitter splitter = new RandomSplitter(); + Table output = splitter.transform(tempTable)[0]; + + assertEquals( + Arrays.asList("test_input", "dummy_input"), + output.getResolvedSchema().getColumnNames()); + } + + @Test + public void testFractions() throws Exception { + Table data = getTable(10000); + RandomSplitter splitter = new RandomSplitter().setFractions(0.4, 0.6); + Table[] output = splitter.transform(data); + + List<Row> result0 = IteratorUtils.toList(tEnv.toDataStream(output[0]).executeAndCollect()); + List<Row> result1 = IteratorUtils.toList(tEnv.toDataStream(output[1]).executeAndCollect()); + List<Row> result2 = IteratorUtils.toList(tEnv.toDataStream(output[2]).executeAndCollect()); + assertEquals(result0.size() / 4000.0, 1.0, 0.1); + assertEquals(result1.size() / 2000.0, 1.0, 0.1); + assertEquals(result2.size() / 4000.0, 1.0, 0.1); Review Comment: Could you please add a comment here, briefly describing the probability that the split result has a deviation of more than 10%? This test case seems might fail from time to time, and I want to be assured that the probability is low enough that it would not affect the CIs. Or would it be better to make the `RandomSplitter` have a fixed behavior in our test cases? This means setting the parallelism to 1 and using a fixed random seed in the algorithm. This also helps to avoid introducing flaky tests into Flink ML. Same for other test cases. ########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/RandomSplitterTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.ml.feature; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.feature.randomsplitter.RandomSplitter; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** Tests {@link RandomSplitter}. */ +public class RandomSplitterTest extends AbstractTestBase { + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(4); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + + tEnv = StreamTableEnvironment.create(env); + } + + private Table getTable(int size) { + List<Row> data = new ArrayList<>(); + for (int i = 0; i < size; ++i) { + data.add(Row.of(i)); + } + return tEnv.fromDataStream(env.fromCollection(data)); Review Comment: nit: `StreamExecutionEnvironment.fromSequence` might be better. ########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/RandomSplitterTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.ml.feature; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.feature.randomsplitter.RandomSplitter; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; + +import org.apache.commons.collections.IteratorUtils; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** Tests {@link RandomSplitter}. */ +public class RandomSplitterTest extends AbstractTestBase { + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(4); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + + tEnv = StreamTableEnvironment.create(env); + } + + private Table getTable(int size) { + List<Row> data = new ArrayList<>(); + for (int i = 0; i < size; ++i) { + data.add(Row.of(i)); + } + return tEnv.fromDataStream(env.fromCollection(data)); + } + + @Test + public void testParam() { + RandomSplitter splitter = new RandomSplitter(); + assertEquals(0.5, splitter.getFractions()[0], 1.0e-5); + + splitter.setFractions(0.3, 0.4); + assertArrayEquals(new Double[] {0.3, 0.4}, splitter.getFractions()); + } + + @Test + public void testOutputSchema() { + Table tempTable = + tEnv.fromDataStream(env.fromElements(Row.of("", ""))) + .as("test_input", "dummy_input"); + + RandomSplitter splitter = new RandomSplitter(); + Table output = splitter.transform(tempTable)[0]; + + assertEquals( + Arrays.asList("test_input", "dummy_input"), + output.getResolvedSchema().getColumnNames()); + } + + @Test + public void testFractions() throws Exception { Review Comment: Let's add a check in each test case that the sum of the number of elements in all output tables is equal to the number of elements in the input table, and that each element in the input table appears once and only once in an output table. ########## flink-ml-examples/src/main/java/org/apache/flink/ml/examples/feature/RandomSplitterExample.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.ml.examples.feature; + +import org.apache.flink.ml.feature.randomsplitter.RandomSplitter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +/** Simple program that creates a Splitter instance and uses it for data splitting. */ Review Comment: nit: `Splitter` --> `RandomSplitter`. Same for other documents and comments. ########## flink-ml-examples/src/main/java/org/apache/flink/ml/examples/feature/RandomSplitterExample.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.ml.examples.feature; + +import org.apache.flink.ml.feature.randomsplitter.RandomSplitter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +/** Simple program that creates a Splitter instance and uses it for data splitting. */ +public class RandomSplitterExample { + public static void main(String[] args) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + // Generates input data. + DataStream<Row> inputStream = + env.fromElements( + Row.of(1, 10, 0), + Row.of(1, 10, 0), + Row.of(1, 10, 0), + Row.of(4, 10, 0), + Row.of(5, 10, 0), + Row.of(6, 10, 0), + Row.of(7, 10, 0), + Row.of(10, 10, 0), + Row.of(13, 10, 3)); + Table inputTable = tEnv.fromDataStream(inputStream).as("input"); + + // Creates a Splitter object and initializes its parameters. + RandomSplitter splitter = new RandomSplitter().setFractions(0.4); + + // Uses the Splitter to split inputData. + Table[] outputTable = splitter.transform(inputTable); + + // Extracts and displays the results. + for (CloseableIterator<Row> it = outputTable[0].execute().collect(); it.hasNext(); ) { + System.out.printf("Data 1 : %s\n", it.next()); Review Comment: How about changing `Data 1:` to `Split Result 1 (40%)`, and moving it out of the for loop? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
