Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Flink: Custom partitioner for bucket partitions #7161

Merged
merged 24 commits into from
Aug 10, 2023
Merged
Show file tree
Hide file tree
Changes from 20 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
/*
* 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.iceberg.flink.sink;

import java.util.stream.IntStream;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.PartitionKey;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.flink.RowDataWrapper;

/**
* A {@link KeySelector} that extracts the bucketId from a data row's bucket partition as the key.
* To be used with the {@link BucketPartitioner}.
*/
class BucketPartitionKeySelector implements KeySelector<RowData, Integer> {

private final Schema schema;
private final PartitionKey partitionKey;
private final RowType flinkSchema;
private final int bucketFieldPosition;

private transient RowDataWrapper rowDataWrapper;

BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) {
kengtin marked this conversation as resolved.
Show resolved Hide resolved
this.schema = schema;
this.partitionKey = new PartitionKey(partitionSpec, schema);
this.flinkSchema = flinkSchema;
int bucketFieldId = BucketPartitionerUtil.getBucketFieldInfo(partitionSpec).f0;
this.bucketFieldPosition =
stevenzwu marked this conversation as resolved.
Show resolved Hide resolved
IntStream.range(0, partitionSpec.fields().size())
.filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId)
.toArray()[0];
}

private RowDataWrapper lazyRowDataWrapper() {
if (rowDataWrapper == null) {
rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct());
}
return rowDataWrapper;
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: iceberg style adds an empty line after the end of a control block }

}

@Override
public Integer getKey(RowData rowData) {
partitionKey.partition(lazyRowDataWrapper().wrap(rowData));
return partitionKey.get(this.bucketFieldPosition, Integer.class);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
/*
* 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.iceberg.flink.sink;

import org.apache.flink.api.common.functions.Partitioner;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;

/**
* This partitioner will redirect records to writers deterministically based on the Bucket partition
* spec. It'll attempt to optimize the file size written depending on whether numPartitions is
* greater, less or equal than the maxNumBuckets. Note: The current implementation only supports ONE
* bucket in the partition spec.
*/
class BucketPartitioner implements Partitioner<Integer> {

static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null";
static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE =
"Invalid bucket ID %s: must be non-negative.";
static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE =
"Invalid bucket ID %s: must be less than bucket limit: %s.";

private final int maxNumBuckets;

// To hold the OFFSET of the next writer to use for any bucket, only used when writers > the
// number of buckets
private final int[] currentBucketWriterOffset;

BucketPartitioner(PartitionSpec partitionSpec) {
Tuple2<Integer, Integer> bucketFieldInfo =
BucketPartitionerUtil.getBucketFieldInfo(partitionSpec);
stevenzwu marked this conversation as resolved.
Show resolved Hide resolved

this.maxNumBuckets = bucketFieldInfo.f1;
this.currentBucketWriterOffset = new int[this.maxNumBuckets];
}

/**
* Determine the partition id based on the following criteria: If the number of writers <= the
* number of buckets, an evenly distributed number of buckets will be assigned to each writer (one
* writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic
* is handled by the {@link #getPartitionWithMoreWritersThanBuckets
* getPartitionWritersGreaterThanBuckets} method.
*
* @param bucketId the bucketId for each request
* @param numPartitions the total number of partitions
* @return the partition id (writer) to use for each request
*/
@Override
public int partition(Integer bucketId, int numPartitions) {
stevenzwu marked this conversation as resolved.
Show resolved Hide resolved
Preconditions.checkNotNull(bucketId, BUCKET_NULL_MESSAGE);
Preconditions.checkArgument(bucketId >= 0, BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, bucketId);
Preconditions.checkArgument(
bucketId < maxNumBuckets, BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, bucketId, maxNumBuckets);

if (numPartitions <= maxNumBuckets) {
return bucketId % numPartitions;
} else {
return getPartitionWithMoreWritersThanBuckets(bucketId, numPartitions);
}
}

/*-
* If the number of writers > the number of buckets each partitioner will keep a state of multiple
stevenzwu marked this conversation as resolved.
Show resolved Hide resolved
* writers per bucket as evenly as possible, and will round-robin the requests across them, in this
* case each writer will target only one bucket at all times (many writers -> one bucket). Example:
* Configuration: numPartitions (writers) = 5, maxBuckets = 2
* Expected behavior:
* - Records for Bucket 0 will be "round robin" between Writers 0, 2 and 4
* - Records for Bucket 1 will always use Writer 1 and 3
* Notes:
* - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId
* - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter).
* In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0.
*
* @return the destination partition index (writer subtask id)
*/
private int getPartitionWithMoreWritersThanBuckets(int bucketId, int numPartitions) {
int currentOffset = currentBucketWriterOffset[bucketId];
// Determine if this bucket requires an "extra writer"
int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0;
// The max number of writers this bucket can have
int maxNumWritersPerBucket = (numPartitions / maxNumBuckets) + extraWriter;

// Increment the writer offset or reset if it's reached the max for this bucket
int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1;
currentBucketWriterOffset[bucketId] = nextOffset;

return bucketId + (maxNumBuckets * currentOffset);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
* 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.iceberg.flink.sink;

import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.transforms.PartitionSpecVisitor;

final class BucketPartitionerUtil {
static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE =
"Unsupported partition spec: bucket partitioner expects 1 bucket partition, received: %s";

private BucketPartitionerUtil() {}

/**
* Determines whether the PartitionSpec has one and only one Bucket definition
*
* @param partitionSpec the partition spec in question
* @return whether the PartitionSpec has only one Bucket
*/
static boolean hasOneBucketField(PartitionSpec partitionSpec) {
List<Tuple2<Integer, Integer>> bucketFields = getBucketFields(partitionSpec);
return bucketFields != null && bucketFields.size() == 1;
}

/**
* Extracts the Bucket definition from a PartitionSpec.
*
* @param partitionSpec the partition spec in question
* @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets)
*/
static Tuple2<Integer, Integer> getBucketFieldInfo(PartitionSpec partitionSpec) {
List<Tuple2<Integer, Integer>> bucketFields = getBucketFields(partitionSpec);
Preconditions.checkArgument(
bucketFields.size() == 1,
BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE,
bucketFields.size());
return bucketFields.get(0);
}

private static List<Tuple2<Integer, Integer>> getBucketFields(PartitionSpec spec) {
return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream()
.filter(Objects::nonNull)
.collect(Collectors.toList());
}

private static class BucketPartitionSpecVisitor
implements PartitionSpecVisitor<Tuple2<Integer, Integer>> {
@Override
public Tuple2<Integer, Integer> identity(int fieldId, String sourceName, int sourceId) {
return null;
}

@Override
public Tuple2<Integer, Integer> bucket(
int fieldId, String sourceName, int sourceId, int numBuckets) {
return new Tuple2<>(fieldId, numBuckets);
}

@Override
public Tuple2<Integer, Integer> truncate(
int fieldId, String sourceName, int sourceId, int width) {
return null;
}

@Override
public Tuple2<Integer, Integer> year(int fieldId, String sourceName, int sourceId) {
return null;
}

@Override
public Tuple2<Integer, Integer> month(int fieldId, String sourceName, int sourceId) {
return null;
}

@Override
public Tuple2<Integer, Integer> day(int fieldId, String sourceName, int sourceId) {
return null;
}

@Override
public Tuple2<Integer, Integer> hour(int fieldId, String sourceName, int sourceId) {
return null;
}

@Override
public Tuple2<Integer, Integer> alwaysNull(int fieldId, String sourceName, int sourceId) {
return null;
}

@Override
public Tuple2<Integer, Integer> unknown(
int fieldId, String sourceName, int sourceId, String transform) {
return null;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -505,7 +505,13 @@ private DataStream<RowData> distributeDataStream(
+ "and table is unpartitioned");
return input;
} else {
return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
if (BucketPartitionerUtil.hasOneBucketField(partitionSpec)) {
return input.partitionCustom(
new BucketPartitioner(partitionSpec),
new BucketPartitionKeySelector(partitionSpec, iSchema, flinkRowType));
} else {
return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
}
}
} else {
if (partitionSpec.isUnpartitioned()) {
Expand Down