zhipeng93 commented on code in PR #160: URL: https://github.com/apache/flink-ml/pull/160#discussion_r1012681894
########## docs/content/docs/operators/feature/randomsplitter.md: ########## @@ -0,0 +1,148 @@ +--- +title: "RandomSplitter" +weight: 1 +type: docs +aliases: +- /operators/feature/randomSplitter.html Review Comment: There should be no capitalized letter here. Let's update it as `/operators/feature/randomsplitter.html`. ########## docs/content/docs/operators/feature/randomsplitter.md: ########## @@ -0,0 +1,148 @@ +--- +title: "RandomSplitter" +weight: 1 +type: docs +aliases: +- /operators/feature/randomSplitter.html +--- + +<!-- +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. +--> + +## RandomSplitter + +An AlgoOperator which splits a table into N tables according to the given weights. + +### Parameters + +| Key | Default | Type | Required | Description | +|:--------|:-------------|:----------|:---------|:-------------------------------| +| weights | `[1.0, 1.0]` | Do uble[] | no | The weights of data splitting. | Review Comment: `Do uble[]` should be updated as `Double[]`, i.e., there should be no space here. ########## docs/content/docs/operators/feature/randomsplitter.md: ########## @@ -0,0 +1,148 @@ +--- +title: "RandomSplitter" +weight: 1 +type: docs +aliases: +- /operators/feature/randomSplitter.html +--- + +<!-- +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. +--> + +## RandomSplitter + +An AlgoOperator which splits a table into N tables according to the given weights. + +### Parameters + +| Key | Default | Type | Required | Description | +|:--------|:-------------|:----------|:---------|:-------------------------------| +| weights | `[1.0, 1.0]` | Do uble[] | no | The weights of data splitting. | + +### Examples + +{{< tabs examples >}} + +{{< tab "Java">}} + +```java +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 RandomSplitter 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().setWeights(4.0, 6.0); + + // Uses the Splitter to split inputData. + Table[] outputTable = splitter.transform(inputTable); + + // Extracts and displays the results. + System.out.println("Split Result 1 (40%)"); + for (CloseableIterator<Row> it = outputTable[0].execute().collect(); it.hasNext(); ) { + System.out.printf("%s\n", it.next()); + } + System.out.println("Split Result 2 (60%)"); + for (CloseableIterator<Row> it = outputTable[1].execute().collect(); it.hasNext(); ) { + System.out.printf("%s\n", it.next()); + } + } +} + +``` + +{{< /tab>}} + +{{< tab "Python">}} + +```python +# Simple program that creates a RandomSplitter instance and uses it for feature Review Comment: use it for feature engineering --> use it for data splitting. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/randomsplitter/RandomSplitter.java: ########## @@ -0,0 +1,128 @@ +/* + * 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.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.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +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.OutputTag; +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.Random; + +/** An AlgoOperator which splits a Table into N Tables according to the given weights. */ +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(); + RowTypeInfo outputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema()); + + final Double[] weights = getWeights(); + OutputTag<Row>[] outputTags = new OutputTag[weights.length - 1]; + for (int i = 0; i < outputTags.length; ++i) { + outputTags[i] = new OutputTag<Row>("outputTag_" + i, outputTypeInfo) {}; + } + + SingleOutputStreamOperator<Row> results = + tEnv.toDataStream(inputs[0]) + .transform( + "SplitterOperator", + outputTypeInfo, + new SplitterOperator(outputTags, weights)); + + Table[] outputTables = new Table[weights.length]; + outputTables[0] = tEnv.fromDataStream(results); + + for (int i = 0; i < outputTags.length; ++i) { + DataStream<Row> dataStream = results.getSideOutput(outputTags[i]); + outputTables[i + 1] = tEnv.fromDataStream(dataStream); + } + return outputTables; + } + + private static class SplitterOperator extends AbstractStreamOperator<Row> + implements OneInputStreamOperator<Row, Row> { + private final Random random = new Random(0); + OutputTag<Row>[] outputTag; + final double[] fractions; + + public SplitterOperator(OutputTag<Row>[] outputTag, Double[] weights) { + this.outputTag = outputTag; + this.fractions = new double[weights.length]; + double weightSum = 0.0; + for (Double weight : weights) { + weightSum += weight; + } + double currentSum = 0.0; + for (int i = 0; i < fractions.length; ++i) { + currentSum += weights[i]; + fractions[i] = currentSum / weightSum; + } + } + + @Override + public void processElement(StreamRecord<Row> streamRecord) throws Exception { + int index = -Arrays.binarySearch(fractions, random.nextDouble()) - 2; Review Comment: Let's also handle the case when the returned value is positive. ########## docs/content/docs/operators/feature/randomsplitter.md: ########## @@ -0,0 +1,148 @@ +--- +title: "RandomSplitter" +weight: 1 +type: docs +aliases: +- /operators/feature/randomSplitter.html +--- + +<!-- +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. +--> + +## RandomSplitter + +An AlgoOperator which splits a table into N tables according to the given weights. + +### Parameters + +| Key | Default | Type | Required | Description | +|:--------|:-------------|:----------|:---------|:-------------------------------| +| weights | `[1.0, 1.0]` | Do uble[] | no | The weights of data splitting. | + +### Examples + +{{< tabs examples >}} + +{{< tab "Java">}} + +```java +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 RandomSplitter 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. Review Comment: Let's update `Splitter` as `RandomSplitter`, same for other cases in this class as well as `RandomSplitterExample` ########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/randomsplitter/RandomSplitterParams.java: ########## @@ -0,0 +1,62 @@ +/* + * 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; + +/** + * Params of {@link RandomSplitter}. + * + * @param <T> The class type of this instance. + */ +public interface RandomSplitterParams<T> extends WithParams<T> { + /** Weights must be a nonempty array with elements greater than zero. */ Review Comment: How about using the following java doc? `Weights should be a non-empty array with all elements greater than zero. The weights will be normalized such that the sum of all elements equals to one.` ########## docs/content/docs/operators/feature/randomsplitter.md: ########## @@ -0,0 +1,148 @@ +--- +title: "RandomSplitter" +weight: 1 +type: docs +aliases: +- /operators/feature/randomSplitter.html +--- + +<!-- +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. +--> + +## RandomSplitter + +An AlgoOperator which splits a table into N tables according to the given weights. + +### Parameters + +| Key | Default | Type | Required | Description | +|:--------|:-------------|:----------|:---------|:-------------------------------| +| weights | `[1.0, 1.0]` | Do uble[] | no | The weights of data splitting. | + +### Examples + +{{< tabs examples >}} + +{{< tab "Java">}} + +```java +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 RandomSplitter 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().setWeights(4.0, 6.0); + + // Uses the Splitter to split inputData. + Table[] outputTable = splitter.transform(inputTable); Review Comment: nit: outputTable --> outputTables ########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/RandomSplitterTest.java: ########## @@ -0,0 +1,150 @@ +/* + * 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.datastream.DataStreamSource; +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.Comparator; +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(1); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + + tEnv = StreamTableEnvironment.create(env); + } + + private Table getTable(int size) { + DataStreamSource<Long> dataStream = env.fromSequence(0L, size); + return tEnv.fromDataStream(dataStream); + } + + @Test + public void testParam() { + RandomSplitter splitter = new RandomSplitter(); + splitter.setWeights(0.3, 0.4); + assertArrayEquals(new Double[] {0.3, 0.4}, splitter.getWeights()); + } + + @Test + public void testOutputSchema() { + Table tempTable = + tEnv.fromDataStream(env.fromElements(Row.of("", ""))) + .as("test_input", "dummy_input"); + + RandomSplitter splitter = new RandomSplitter().setWeights(0.5, 0.1); + Table[] output = splitter.transform(tempTable); + assertEquals(2, output.length); + for (Table table : output) { + assertEquals( + Arrays.asList("test_input", "dummy_input"), + table.getResolvedSchema().getColumnNames()); + } + } + + @Test + public void testWeights() throws Exception { + Table data = getTable(1000); + RandomSplitter splitter = new RandomSplitter().setWeights(2.0, 1.0, 2.0); + 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() / 400.0, 1.0, 0.1); + assertEquals(result1.size() / 200.0, 1.0, 0.1); + assertEquals(result2.size() / 400.0, 1.0, 0.1); + verifyResultTables(data, output); + } + + @Test + public void testSaveLoadAndTransform() throws Exception { + Table data = getTable(2000); + RandomSplitter randomSplitter = new RandomSplitter().setWeights(4.0, 6.0); + + RandomSplitter splitterLoad = + TestUtils.saveAndReload( + tEnv, randomSplitter, TEMPORARY_FOLDER.newFolder().getAbsolutePath()); + + Table[] output = splitterLoad.transform(data); + List<Row> result0 = IteratorUtils.toList(tEnv.toDataStream(output[0]).executeAndCollect()); + List<Row> result1 = IteratorUtils.toList(tEnv.toDataStream(output[1]).executeAndCollect()); + assertEquals(result0.size() / 800.0, 1.0, 0.1); + assertEquals(result1.size() / 1200.0, 1.0, 0.1); + verifyResultTables(data, output); + } + + Comparator<Row> comparator = + new Comparator<Row>() { + @Override + public int compare(Row o1, Row o2) { + long value1 = o1.getFieldAs(0); + long value2 = o2.getFieldAs(0); + return Long.compare(value1, value2); + } + }; + + private void verifyResultTables(Table input, Table[] output) throws Exception { + List<Row> expectedData = IteratorUtils.toList(tEnv.toDataStream(input).executeAndCollect()); + List<Row> results = new ArrayList<>(); + for (Table table : output) { + List<Row> result = IteratorUtils.toList(tEnv.toDataStream(table).executeAndCollect()); + results.addAll(result); + } + expectedData.sort(comparator); Review Comment: Let's simplify the logic with `TestbaseUtils.CompareResultCollections` ########## flink-ml-python/pyflink/ml/lib/feature/normalizer.py: ########## @@ -44,7 +44,7 @@ def __init__(self, java_params): def set_p(self, value: float): return typing.cast(_NormalizerParams, self.set(self.P, value)) - def get_p(self) -> bool: + def get_p(self) -> float: Review Comment: Is there a unit test for the bugfix? -- 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]
