[
https://issues.apache.org/jira/browse/FLINK-1901?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14693288#comment-14693288
]
ASF GitHub Bot commented on FLINK-1901:
---------------------------------------
Github user thvasilo commented on a diff in the pull request:
https://github.com/apache/flink/pull/949#discussion_r36844879
--- Diff:
flink-core/src/test/java/org/apache/flink/api/common/operators/util/RandomSamplerTest.java
---
@@ -0,0 +1,425 @@
+/*
+ * 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.api.common.operators.util;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * This test suite try to verify whether all the random samplers work as
we expected, which mainly focus on:
+ * <ul>
+ * <li>Does sampled result fit into input parameters? we check parameters
like sample fraction, sample size,
+ * w/o replacement, and so on.</li>
+ * <li>Does sampled result randomly selected? we verify this by measure
how much does it distributed on source data.
+ * Run Kolmogorov-Smirnov (KS) test between the random samplers and
default reference samplers which is distributed
+ * well-proportioned on source data. If random sampler select elements
randomly from source, it would distributed
+ * well-proportioned on source data as well. The KS test will fail to
strongly reject the null hypothesis that
+ * the distributions of sampling gaps are the same.
+ * </li>
+ * </ul>
+ *
+ * @see <a
href="https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test">Kolmogorov
Smirnov test</a>
+ */
+public class RandomSamplerTest {
+ private final static int SOURCE_SIZE = 10000;
+ private static KolmogorovSmirnovTest ksTest;
+ private static List<Double> source;
+ private final static int DEFFAULT_PARTITION_NUMBER=10;
+ private List<Double>[] sourcePartitions = new
List[DEFFAULT_PARTITION_NUMBER];
+
+ @BeforeClass
+ public static void init() {
+ // initiate source data set.
+ source = new ArrayList<Double>(SOURCE_SIZE);
+ for (int i = 0; i < SOURCE_SIZE; i++) {
+ source.add((double) i);
+ }
+
+ ksTest = new KolmogorovSmirnovTest();
+ }
+
+ private void initSourcePartition() {
+ for (int i=0; i<DEFFAULT_PARTITION_NUMBER; i++) {
+ sourcePartitions[i] = new LinkedList<Double>();
+ }
+ for (int i = 0; i< SOURCE_SIZE; i++) {
+ int index = i % DEFFAULT_PARTITION_NUMBER;
+ sourcePartitions[index].add((double)i);
+ }
+ }
+
+ @Test(expected = java.lang.IllegalArgumentException.class)
+ public void testBernoulliSamplerWithUnexpectedFraction1() {
+ verifySamplerFraction(-1, false);
+ }
+
+ @Test(expected = java.lang.IllegalArgumentException.class)
+ public void testBernoulliSamplerWithUnexpectedFraction2() {
+ verifySamplerFraction(2, false);
+ }
+
+ @Test
+ public void testBernoulliSamplerFraction() {
+ verifySamplerFraction(0.01, false);
+ verifySamplerFraction(0.05, false);
+ verifySamplerFraction(0.1, false);
+ verifySamplerFraction(0.3, false);
+ verifySamplerFraction(0.5, false);
+ verifySamplerFraction(0.854, false);
+ verifySamplerFraction(0.99, false);
+ }
+
+ @Test
+ public void testBernoulliSamplerDuplicateElements() {
+ verifyRandomSamplerDuplicateElements(new
BernoulliSampler<Double>(0.01));
+ verifyRandomSamplerDuplicateElements(new
BernoulliSampler<Double>(0.1));
+ verifyRandomSamplerDuplicateElements(new
BernoulliSampler<Double>(0.5));
+ }
+
+ @Test(expected = java.lang.IllegalArgumentException.class)
+ public void testPoissonSamplerWithUnexpectedFraction1() {
+ verifySamplerFraction(-1, true);
+ }
+
+ @Test
+ public void testPoissonSamplerFraction() {
+ verifySamplerFraction(0.01, true);
+ verifySamplerFraction(0.05, true);
+ verifySamplerFraction(0.1, true);
+ verifySamplerFraction(0.5, true);
+ verifySamplerFraction(0.854, true);
+ verifySamplerFraction(0.99, true);
+ verifySamplerFraction(1.5, true);
+ }
+
+ @Test(expected = java.lang.IllegalArgumentException.class)
+ public void testReservoirSamplerUnexpectedSize1() {
+ verifySamplerFixedSampleSize(-1, true);
+ }
+
+ @Test(expected = java.lang.IllegalArgumentException.class)
+ public void testReservoirSamplerUnexpectedSize2() {
+ verifySamplerFixedSampleSize(-1, false);
+ }
+
+ @Test
+ public void testBernoulliSamplerDistribution() {
+ verifyBernoulliSampler(0.01d);
+ verifyBernoulliSampler(0.05d);
+ verifyBernoulliSampler(0.1d);
+ verifyBernoulliSampler(0.5d);
+ }
+
+ @Test
+ public void testPoissonSamplerDistribution() {
+ verifyPoissonSampler(0.01d);
+ verifyPoissonSampler(0.05d);
+ verifyPoissonSampler(0.1d);
+ verifyPoissonSampler(0.5d);
+ }
+
+ @Test
+ public void testReservoirSamplerSampledSize() {
+ verifySamplerFixedSampleSize(1, true);
+ verifySamplerFixedSampleSize(10, true);
+ verifySamplerFixedSampleSize(100, true);
+ verifySamplerFixedSampleSize(1234, true);
+ verifySamplerFixedSampleSize(9999, true);
+ verifySamplerFixedSampleSize(20000, true);
+
+ verifySamplerFixedSampleSize(1, false);
+ verifySamplerFixedSampleSize(10, false);
+ verifySamplerFixedSampleSize(100, false);
+ verifySamplerFixedSampleSize(1234, false);
+ verifySamplerFixedSampleSize(9999, false);
+ }
+
+ @Test
+ public void testReservoirSamplerSampledSize2() {
+ RandomSampler<Double> sampler = new
ReservoirSamplerWithoutReplacement<Double>(20000);
+ Iterator<Double> sampled = sampler.sample(source.iterator());
+ assertTrue("ReservoirSamplerWithoutReplacement sampled output
size should not beyond the source size.", getSize(sampled) == SOURCE_SIZE);
+ }
+
+ @Test
+ public void testReservoirSamplerDuplicateElements() {
+ verifyRandomSamplerDuplicateElements(new
ReservoirSamplerWithoutReplacement<Double>(100));
+ verifyRandomSamplerDuplicateElements(new
ReservoirSamplerWithoutReplacement<Double>(1000));
+ verifyRandomSamplerDuplicateElements(new
ReservoirSamplerWithoutReplacement<Double>(5000));
+ }
+
+ @Test
+ public void testReservoirSamplerWithoutReplacement() {
+ verifyReservoirSamplerWithoutReplacement(100, false);
+ verifyReservoirSamplerWithoutReplacement(500, false);
+ verifyReservoirSamplerWithoutReplacement(1000, false);
+ verifyReservoirSamplerWithoutReplacement(5000, false);
+ }
+
+ @Test
+ public void testReservoirSamplerWithReplacement() {
+ verifyReservoirSamplerWithReplacement(100, false);
+ verifyReservoirSamplerWithReplacement(500, false);
+ verifyReservoirSamplerWithReplacement(1000, false);
+ verifyReservoirSamplerWithReplacement(5000, false);
+ }
+
+ @Test
+ public void testReservoirSamplerWithMultiSourcePartitions1() {
+ initSourcePartition();
+
+ verifyReservoirSamplerWithoutReplacement(100, true);
+ verifyReservoirSamplerWithoutReplacement(500, true);
+ verifyReservoirSamplerWithoutReplacement(1000, true);
+ verifyReservoirSamplerWithoutReplacement(5000, true);
+ }
+
+ @Test
+ public void testReservoirSamplerWithMultiSourcePartitions2() {
+ initSourcePartition();
+
+ verifyReservoirSamplerWithReplacement(100, true);
+ verifyReservoirSamplerWithReplacement(500, true);
+ verifyReservoirSamplerWithReplacement(1000, true);
+ verifyReservoirSamplerWithReplacement(5000, true);
+ }
+
+ private void verifySamplerFixedSampleSize(int numSample, boolean
withReplacement) {
--- End diff --
Having some comments for some of the ``verify[..]`` functions explaining
what we verify at each would help with code understanding.
> Create sample operator for Dataset
> ----------------------------------
>
> Key: FLINK-1901
> URL: https://issues.apache.org/jira/browse/FLINK-1901
> Project: Flink
> Issue Type: Improvement
> Components: Core
> Reporter: Theodore Vasiloudis
> Assignee: Chengxiang Li
>
> In order to be able to implement Stochastic Gradient Descent and a number of
> other machine learning algorithms we need to have a way to take a random
> sample from a Dataset.
> We need to be able to sample with or without replacement from the Dataset,
> choose the relative or exact size of the sample, set a seed for
> reproducibility, and support sampling within iterations.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)