/
DataSetUtils.java
460 lines (418 loc) · 22.5 KB
/
DataSetUtils.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
/*
* 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.java.utils;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.distributions.DataDistribution;
import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
import org.apache.flink.api.common.functions.MapPartitionFunction;
import org.apache.flink.api.common.functions.OpenContext;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.functions.RichMapPartitionFunction;
import org.apache.flink.api.common.operators.Keys;
import org.apache.flink.api.common.operators.base.PartitionOperatorBase;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.functions.SampleInCoordinator;
import org.apache.flink.api.java.functions.SampleInPartition;
import org.apache.flink.api.java.functions.SampleWithFraction;
import org.apache.flink.api.java.operators.GroupReduceOperator;
import org.apache.flink.api.java.operators.MapPartitionOperator;
import org.apache.flink.api.java.operators.PartitionOperator;
import org.apache.flink.api.java.summarize.aggregation.SummaryAggregatorFactory;
import org.apache.flink.api.java.summarize.aggregation.TupleSummaryAggregator;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.util.AbstractID;
import org.apache.flink.util.Collector;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
/**
* This class provides simple utility methods for zipping elements in a data set with an index or
* with a unique identifier.
*
* @deprecated All Flink DataSet APIs are deprecated since Flink 1.18 and will be removed in a
* future Flink major version. You can still build your application in DataSet, but you should
* move to either the DataStream and/or Table API.
* @see <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741">
* FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API</a>
*/
@Deprecated
@PublicEvolving
public final class DataSetUtils {
/**
* Method that goes over all the elements in each partition in order to retrieve the total
* number of elements.
*
* @param input the DataSet received as input
* @return a data set containing tuples of subtask index, number of elements mappings.
*/
public static <T> DataSet<Tuple2<Integer, Long>> countElementsPerPartition(DataSet<T> input) {
return input.mapPartition(
new RichMapPartitionFunction<T, Tuple2<Integer, Long>>() {
@Override
public void mapPartition(
Iterable<T> values, Collector<Tuple2<Integer, Long>> out)
throws Exception {
long counter = 0;
for (T value : values) {
counter++;
}
out.collect(
new Tuple2<>(
getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(),
counter));
}
});
}
/**
* Method that assigns a unique {@link Long} value to all elements in the input data set. The
* generated values are consecutive.
*
* @param input the input data set
* @return a data set of tuple 2 consisting of consecutive ids and initial values.
*/
public static <T> DataSet<Tuple2<Long, T>> zipWithIndex(DataSet<T> input) {
DataSet<Tuple2<Integer, Long>> elementCount = countElementsPerPartition(input);
return input.mapPartition(
new RichMapPartitionFunction<T, Tuple2<Long, T>>() {
long start = 0;
@Override
public void open(OpenContext openContext) throws Exception {
super.open(openContext);
List<Tuple2<Integer, Long>> offsets =
getRuntimeContext()
.getBroadcastVariableWithInitializer(
"counts",
new BroadcastVariableInitializer<
Tuple2<Integer, Long>,
List<Tuple2<Integer, Long>>>() {
@Override
public List<Tuple2<Integer, Long>>
initializeBroadcastVariable(
Iterable<
Tuple2<
Integer,
Long>>
data) {
// sort the list by task id to
// calculate the correct offset
List<Tuple2<Integer, Long>>
sortedData =
new ArrayList<>();
for (Tuple2<Integer, Long> datum :
data) {
sortedData.add(datum);
}
Collections.sort(
sortedData,
new Comparator<
Tuple2<
Integer,
Long>>() {
@Override
public int compare(
Tuple2<
Integer,
Long>
o1,
Tuple2<
Integer,
Long>
o2) {
return o1.f0
.compareTo(
o2.f0);
}
});
return sortedData;
}
});
// compute the offset for each partition
for (int i = 0;
i
< getRuntimeContext()
.getTaskInfo()
.getIndexOfThisSubtask();
i++) {
start += offsets.get(i).f1;
}
}
@Override
public void mapPartition(
Iterable<T> values, Collector<Tuple2<Long, T>> out)
throws Exception {
for (T value : values) {
out.collect(new Tuple2<>(start++, value));
}
}
})
.withBroadcastSet(elementCount, "counts");
}
/**
* Method that assigns a unique {@link Long} value to all elements in the input data set as
* described below.
*
* <ul>
* <li>a map function is applied to the input data set
* <li>each map task holds a counter c which is increased for each record
* <li>c is shifted by n bits where n = log2(number of parallel tasks)
* <li>to create a unique ID among all tasks, the task id is added to the counter
* <li>for each record, the resulting counter is collected
* </ul>
*
* @param input the input data set
* @return a data set of tuple 2 consisting of ids and initial values.
*/
public static <T> DataSet<Tuple2<Long, T>> zipWithUniqueId(DataSet<T> input) {
return input.mapPartition(
new RichMapPartitionFunction<T, Tuple2<Long, T>>() {
long maxBitSize = getBitSize(Long.MAX_VALUE);
long shifter = 0;
long start = 0;
long taskId = 0;
long label = 0;
@Override
public void open(OpenContext openContext) throws Exception {
super.open(openContext);
shifter =
getBitSize(
getRuntimeContext()
.getTaskInfo()
.getNumberOfParallelSubtasks()
- 1);
taskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();
}
@Override
public void mapPartition(Iterable<T> values, Collector<Tuple2<Long, T>> out)
throws Exception {
for (T value : values) {
label = (start << shifter) + taskId;
if (getBitSize(start) + shifter < maxBitSize) {
out.collect(new Tuple2<>(label, value));
start++;
} else {
throw new Exception(
"Exceeded Long value range while generating labels");
}
}
}
});
}
// --------------------------------------------------------------------------------------------
// Sample
// --------------------------------------------------------------------------------------------
/**
* Generate a sample of DataSet by the probability fraction of each element.
*
* @param withReplacement Whether element can be selected more than once.
* @param fraction Probability that each element is chosen, should be [0,1] without replacement,
* and [0, ∞) with replacement. While fraction is larger than 1, the elements are expected
* to be selected multi times into sample on average.
* @return The sampled DataSet
*/
public static <T> MapPartitionOperator<T, T> sample(
DataSet<T> input, final boolean withReplacement, final double fraction) {
return sample(input, withReplacement, fraction, Utils.RNG.nextLong());
}
/**
* Generate a sample of DataSet by the probability fraction of each element.
*
* @param withReplacement Whether element can be selected more than once.
* @param fraction Probability that each element is chosen, should be [0,1] without replacement,
* and [0, ∞) with replacement. While fraction is larger than 1, the elements are expected
* to be selected multi times into sample on average.
* @param seed random number generator seed.
* @return The sampled DataSet
*/
public static <T> MapPartitionOperator<T, T> sample(
DataSet<T> input,
final boolean withReplacement,
final double fraction,
final long seed) {
return input.mapPartition(new SampleWithFraction<T>(withReplacement, fraction, seed));
}
/**
* Generate a sample of DataSet which contains fixed size elements.
*
* <p><strong>NOTE:</strong> Sample with fixed size is not as efficient as sample with fraction,
* use sample with fraction unless you need exact precision.
*
* @param withReplacement Whether element can be selected more than once.
* @param numSamples The expected sample size.
* @return The sampled DataSet
*/
public static <T> DataSet<T> sampleWithSize(
DataSet<T> input, final boolean withReplacement, final int numSamples) {
return sampleWithSize(input, withReplacement, numSamples, Utils.RNG.nextLong());
}
/**
* Generate a sample of DataSet which contains fixed size elements.
*
* <p><strong>NOTE:</strong> Sample with fixed size is not as efficient as sample with fraction,
* use sample with fraction unless you need exact precision.
*
* @param withReplacement Whether element can be selected more than once.
* @param numSamples The expected sample size.
* @param seed Random number generator seed.
* @return The sampled DataSet
*/
public static <T> DataSet<T> sampleWithSize(
DataSet<T> input,
final boolean withReplacement,
final int numSamples,
final long seed) {
SampleInPartition<T> sampleInPartition =
new SampleInPartition<>(withReplacement, numSamples, seed);
MapPartitionOperator mapPartitionOperator = input.mapPartition(sampleInPartition);
// There is no previous group, so the parallelism of GroupReduceOperator is always 1.
String callLocation = Utils.getCallLocationName();
SampleInCoordinator<T> sampleInCoordinator =
new SampleInCoordinator<>(withReplacement, numSamples, seed);
return new GroupReduceOperator<>(
mapPartitionOperator, input.getType(), sampleInCoordinator, callLocation);
}
// --------------------------------------------------------------------------------------------
// Partition
// --------------------------------------------------------------------------------------------
/** Range-partitions a DataSet on the specified tuple field positions. */
public static <T> PartitionOperator<T> partitionByRange(
DataSet<T> input, DataDistribution distribution, int... fields) {
return new PartitionOperator<>(
input,
PartitionOperatorBase.PartitionMethod.RANGE,
new Keys.ExpressionKeys<>(fields, input.getType(), false),
distribution,
Utils.getCallLocationName());
}
/** Range-partitions a DataSet on the specified fields. */
public static <T> PartitionOperator<T> partitionByRange(
DataSet<T> input, DataDistribution distribution, String... fields) {
return new PartitionOperator<>(
input,
PartitionOperatorBase.PartitionMethod.RANGE,
new Keys.ExpressionKeys<>(fields, input.getType()),
distribution,
Utils.getCallLocationName());
}
/** Range-partitions a DataSet using the specified key selector function. */
public static <T, K extends Comparable<K>> PartitionOperator<T> partitionByRange(
DataSet<T> input, DataDistribution distribution, KeySelector<T, K> keyExtractor) {
final TypeInformation<K> keyType =
TypeExtractor.getKeySelectorTypes(keyExtractor, input.getType());
return new PartitionOperator<>(
input,
PartitionOperatorBase.PartitionMethod.RANGE,
new Keys.SelectorFunctionKeys<>(
input.clean(keyExtractor), input.getType(), keyType),
distribution,
Utils.getCallLocationName());
}
// --------------------------------------------------------------------------------------------
// Summarize
// --------------------------------------------------------------------------------------------
/**
* Summarize a DataSet of Tuples by collecting single pass statistics for all columns.
*
* <p>Example usage:
*
* <pre>{@code
* Dataset<Tuple3<Double, String, Boolean>> input = // [...]
* Tuple3<NumericColumnSummary,StringColumnSummary, BooleanColumnSummary> summary = DataSetUtils.summarize(input)
*
* summary.f0.getStandardDeviation()
* summary.f1.getMaxLength()
* }</pre>
*
* @return the summary as a Tuple the same width as input rows
*/
public static <R extends Tuple, T extends Tuple> R summarize(DataSet<T> input)
throws Exception {
if (!input.getType().isTupleType()) {
throw new IllegalArgumentException(
"summarize() is only implemented for DataSet's of Tuples");
}
final TupleTypeInfoBase<?> inType = (TupleTypeInfoBase<?>) input.getType();
DataSet<TupleSummaryAggregator<R>> result =
input.mapPartition(
new MapPartitionFunction<T, TupleSummaryAggregator<R>>() {
@Override
public void mapPartition(
Iterable<T> values,
Collector<TupleSummaryAggregator<R>> out)
throws Exception {
TupleSummaryAggregator<R> aggregator =
SummaryAggregatorFactory.create(inType);
for (Tuple value : values) {
aggregator.aggregate(value);
}
out.collect(aggregator);
}
})
.reduce(
new ReduceFunction<TupleSummaryAggregator<R>>() {
@Override
public TupleSummaryAggregator<R> reduce(
TupleSummaryAggregator<R> agg1,
TupleSummaryAggregator<R> agg2)
throws Exception {
agg1.combine(agg2);
return agg1;
}
});
return result.collect().get(0).result();
}
// --------------------------------------------------------------------------------------------
// Checksum
// --------------------------------------------------------------------------------------------
/**
* Convenience method to get the count (number of elements) of a DataSet as well as the checksum
* (sum over element hashes).
*
* @return A ChecksumHashCode that represents the count and checksum of elements in the data
* set.
* @deprecated This method will be removed at some point.
*/
@Deprecated
public static <T> Utils.ChecksumHashCode checksumHashCode(DataSet<T> input) throws Exception {
final String id = new AbstractID().toString();
input.output(new Utils.ChecksumHashCodeHelper<T>(id)).name("ChecksumHashCode");
JobExecutionResult res = input.getExecutionEnvironment().execute();
return res.<Utils.ChecksumHashCode>getAccumulatorResult(id);
}
// *************************************************************************
// UTIL METHODS
// *************************************************************************
public static int getBitSize(long value) {
if (value > Integer.MAX_VALUE) {
return 64 - Integer.numberOfLeadingZeros((int) (value >> 32));
} else {
return 32 - Integer.numberOfLeadingZeros((int) value);
}
}
/** Private constructor to prevent instantiation. */
private DataSetUtils() {
throw new RuntimeException();
}
}