Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.beam.sdk.coders.ListCoder;
import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn;
import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.util.WeightedValue;
import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
import org.apache.beam.sdk.values.KV;
Expand Down Expand Up @@ -359,6 +360,13 @@ public Coder<QuantileState<T, ComparatorT>> getAccumulatorCoder(
CoderRegistry registry, Coder<T> elementCoder) {
return new QuantileStateCoder<>(compareFn, elementCoder);
}

@Override
public void populateDisplayData(DisplayData.Builder builder) {
builder
.add("numQuantiles", numQuantiles)
.add("comparer", compareFn.getClass());
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,13 +24,16 @@
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.SerializableCoder;
import org.apache.beam.sdk.transforms.Combine.CombineFn;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;

import com.google.common.hash.Hashing;
import com.google.common.hash.HashingOutputStream;
import com.google.common.io.ByteStreams;

import org.apache.avro.reflect.Nullable;

import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
Expand Down Expand Up @@ -167,6 +170,12 @@ static class Globally<T> extends PTransform<PCollection<T>, PCollection<Long>> {
*/
private final long sampleSize;

/**
* The desired maximum estimation error or null if not specified.
*/
@Nullable
private final Double maximumEstimationError;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Nullable


/**
* @see ApproximateUnique#globally(int)
*/
Expand All @@ -178,7 +187,9 @@ public Globally(int sampleSize) {
+ "In general, the estimation "
+ "error is about 2 / sqrt(sampleSize).");
}

this.sampleSize = sampleSize;
this.maximumEstimationError = null;
}

/**
Expand All @@ -190,7 +201,9 @@ public Globally(double maximumEstimationError) {
"ApproximateUnique needs an "
+ "estimation error between 1% (0.01) and 50% (0.5).");
}

this.sampleSize = sampleSizeFromEstimationError(maximumEstimationError);
this.maximumEstimationError = maximumEstimationError;
}

@Override
Expand All @@ -200,6 +213,11 @@ public PCollection<Long> apply(PCollection<T> input) {
Combine.globally(
new ApproximateUniqueCombineFn<>(sampleSize, coder)));
}

@Override
public void populateDisplayData(DisplayData.Builder builder) {
ApproximateUnique.populateDisplayData(builder, sampleSize, maximumEstimationError);
}
}

/**
Expand All @@ -213,8 +231,18 @@ public PCollection<Long> apply(PCollection<T> input) {
static class PerKey<K, V>
extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Long>>> {

/**
* The number of entries in the statistical sample; the higher this number,
* the more accurate the estimate will be.
*/
private final long sampleSize;

/**
* The the desired maximum estimation error or null if not specified.
*/
@Nullable
private final Double maximumEstimationError;

/**
* @see ApproximateUnique#perKey(int)
*/
Expand All @@ -225,7 +253,9 @@ public PerKey(int sampleSize) {
+ "sampleSize >= 16 for an estimation error <= 50%. In general, "
+ "the estimation error is about 2 / sqrt(sampleSize).");
}

this.sampleSize = sampleSize;
this.maximumEstimationError = null;
}

/**
Expand All @@ -237,7 +267,9 @@ public PerKey(double estimationError) {
"ApproximateUnique.PerKey needs an "
+ "estimation error between 1% (0.01) and 50% (0.5).");
}

this.sampleSize = sampleSizeFromEstimationError(estimationError);
this.maximumEstimationError = estimationError;
}

@Override
Expand All @@ -254,6 +286,11 @@ public PCollection<KV<K, Long>> apply(PCollection<KV<K, V>> input) {
Combine.perKey(new ApproximateUniqueCombineFn<>(
sampleSize, coder).<K>asKeyedFn()));
}

@Override
public void populateDisplayData(DisplayData.Builder builder) {
ApproximateUnique.populateDisplayData(builder, sampleSize, maximumEstimationError);
}
}


Expand Down Expand Up @@ -418,4 +455,11 @@ static <T> long hash(T element, Coder<T> coder) throws CoderException, IOExcepti
static long sampleSizeFromEstimationError(double estimationError) {
return Math.round(Math.ceil(4.0 / Math.pow(estimationError, 2.0)));
}

private static void populateDisplayData(
DisplayData.Builder builder, long sampleSize, Double maxEstimationError) {
builder
.add("sampleSize", sampleSize)
.addIfNotNull("maximumEstimationError", maxEstimationError);
}
}
Loading