Skip to content
Merged
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 @@ -23,7 +23,7 @@
import org.apache.paimon.data.GenericRow;
import org.apache.paimon.data.serializer.InternalRowSerializer;
import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableSet;
import org.apache.paimon.table.sink.BucketComputer;
import org.apache.paimon.table.sink.KeyAndBucketExtractor;
import org.apache.paimon.types.RowType;

import javax.annotation.concurrent.ThreadSafe;
Expand Down Expand Up @@ -71,7 +71,7 @@ int[] hashCodes() {
Set<Integer> createBucketSet(int numBucket) {
ImmutableSet.Builder<Integer> builder = new ImmutableSet.Builder<>();
for (int hash : hashCodes) {
builder.add(BucketComputer.bucket(hash, numBucket));
builder.add(KeyAndBucketExtractor.bucket(hash, numBucket));
}
return builder.build();
}
Expand Down Expand Up @@ -139,7 +139,7 @@ public static Optional<BucketSelector> create(

private static int hash(List<Object> columns, InternalRowSerializer serializer) {
BinaryRow binaryRow = serializer.toBinaryRow(GenericRow.of(columns.toArray()));
return BucketComputer.hashcode(binaryRow);
return KeyAndBucketExtractor.bucketKeyHashCode(binaryRow);
}

private static void assembleRows(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,8 +131,19 @@ public Map<String, String> options() {
return options;
}

public List<String> bucketKeys() {
List<String> bucketKeys = originalBucketKeys();
if (bucketKeys.isEmpty()) {
bucketKeys = trimmedPrimaryKeys();
}
if (bucketKeys.isEmpty()) {
bucketKeys = fieldNames();
}
return bucketKeys;
}

/** Original bucket keys, maybe empty. */
public List<String> originalBucketKeys() {
private List<String> originalBucketKeys() {
String key = options.get(BUCKET_KEY.key());
if (StringUtils.isNullOrWhitespaceOnly(key)) {
return Collections.emptyList();
Expand Down Expand Up @@ -178,14 +189,7 @@ public RowType logicalPartitionType() {
}

public RowType logicalBucketKeyType() {
List<String> bucketKeys = originalBucketKeys();
if (bucketKeys.isEmpty()) {
bucketKeys = trimmedPrimaryKeys();
}
if (bucketKeys.isEmpty()) {
bucketKeys = fieldNames();
}
return projectedLogicalRowType(bucketKeys);
return projectedLogicalRowType(bucketKeys());
}

public RowType logicalTrimmedPrimaryKeysType() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
import org.apache.paimon.predicate.Predicate;
import org.apache.paimon.reader.RecordReader;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.sink.SinkRecordConverter;
import org.apache.paimon.table.sink.InternalRowKeyAndBucketExtractor;
import org.apache.paimon.table.sink.TableWriteImpl;
import org.apache.paimon.table.source.AppendOnlySplitGenerator;
import org.apache.paimon.table.source.DataSplit;
Expand Down Expand Up @@ -125,7 +125,7 @@ public RecordReader<InternalRow> createReader(Split split) throws IOException {
public TableWriteImpl<InternalRow> newWrite(String commitUser) {
return new TableWriteImpl<>(
store().newWrite(commitUser),
new SinkRecordConverter(tableSchema),
new InternalRowKeyAndBucketExtractor(tableSchema),
record -> {
Preconditions.checkState(
record.row().getRowKind() == RowKind.INSERT,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import org.apache.paimon.schema.KeyValueFieldsExtractor;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.stats.BinaryTableStats;
import org.apache.paimon.table.sink.SinkRecordConverter;
import org.apache.paimon.table.sink.InternalRowKeyAndBucketExtractor;
import org.apache.paimon.table.sink.TableWriteImpl;
import org.apache.paimon.table.source.InnerTableRead;
import org.apache.paimon.table.source.KeyValueTableRead;
Expand Down Expand Up @@ -144,7 +144,7 @@ public TableWriteImpl<KeyValue> newWrite(String commitUser) {
final KeyValue kv = new KeyValue();
return new TableWriteImpl<>(
store().newWrite(commitUser),
new SinkRecordConverter(tableSchema),
new InternalRowKeyAndBucketExtractor(tableSchema),
record -> {
switch (record.row().getRowKind()) {
case INSERT:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,8 +38,8 @@
import org.apache.paimon.reader.RecordReader;
import org.apache.paimon.schema.KeyValueFieldsExtractor;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.sink.InternalRowKeyAndBucketExtractor;
import org.apache.paimon.table.sink.SequenceGenerator;
import org.apache.paimon.table.sink.SinkRecordConverter;
import org.apache.paimon.table.sink.TableWriteImpl;
import org.apache.paimon.table.source.InnerTableRead;
import org.apache.paimon.table.source.KeyValueTableRead;
Expand Down Expand Up @@ -222,7 +222,7 @@ public TableWriteImpl<KeyValue> newWrite(String commitUser) {
final KeyValue kv = new KeyValue();
return new TableWriteImpl<>(
store().newWrite(commitUser),
new SinkRecordConverter(tableSchema),
new InternalRowKeyAndBucketExtractor(tableSchema),
record -> {
long sequenceNumber =
sequenceGenerator == null
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
/*
* 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.paimon.table.sink;

import org.apache.paimon.CoreOptions;
import org.apache.paimon.codegen.CodeGenUtils;
import org.apache.paimon.codegen.Projection;
import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.schema.TableSchema;

/** {@link KeyAndBucketExtractor} for {@link InternalRow}. */
public class InternalRowKeyAndBucketExtractor implements KeyAndBucketExtractor<InternalRow> {

private final int numBuckets;
private final boolean sameBucketKeyAndTrimmedPrimaryKey;

private final Projection partitionProjection;
private final Projection bucketKeyProjection;
private final Projection trimmedPrimaryKeyProjection;
private final Projection logPrimaryKeyProjection;

private InternalRow record;

private BinaryRow partition;
private BinaryRow bucketKey;
private Integer bucket;
private BinaryRow trimmedPrimaryKey;
private BinaryRow logPrimaryKey;

public InternalRowKeyAndBucketExtractor(TableSchema schema) {
numBuckets = new CoreOptions(schema.options()).bucket();
sameBucketKeyAndTrimmedPrimaryKey = schema.bucketKeys().equals(schema.trimmedPrimaryKeys());

partitionProjection =
CodeGenUtils.newProjection(
schema.logicalRowType(), schema.projection(schema.partitionKeys()));
bucketKeyProjection =
CodeGenUtils.newProjection(
schema.logicalRowType(), schema.projection(schema.bucketKeys()));
trimmedPrimaryKeyProjection =
CodeGenUtils.newProjection(
schema.logicalRowType(), schema.projection(schema.trimmedPrimaryKeys()));
logPrimaryKeyProjection =
CodeGenUtils.newProjection(
schema.logicalRowType(), schema.projection(schema.primaryKeys()));
}

@Override
public void setRecord(InternalRow record) {
this.record = record;

this.partition = null;
this.bucketKey = null;
this.bucket = null;
this.trimmedPrimaryKey = null;
this.logPrimaryKey = null;
}

@Override
public BinaryRow partition() {
if (partition == null) {
partition = partitionProjection.apply(record);
}
return partition;
}

@Override
public int bucket() {
if (bucketKey == null) {
bucketKey = bucketKeyProjection.apply(record);
if (sameBucketKeyAndTrimmedPrimaryKey) {
trimmedPrimaryKey = bucketKey;
}
}
if (bucket == null) {
bucket =
KeyAndBucketExtractor.bucket(
KeyAndBucketExtractor.bucketKeyHashCode(bucketKey), numBuckets);
}
return bucket;
}

@Override
public BinaryRow trimmedPrimaryKey() {
if (trimmedPrimaryKey == null) {
trimmedPrimaryKey = trimmedPrimaryKeyProjection.apply(record);
if (sameBucketKeyAndTrimmedPrimaryKey) {
bucketKey = trimmedPrimaryKey;
}
}
return trimmedPrimaryKey;
}

@Override
public BinaryRow logPrimaryKey() {
if (logPrimaryKey == null) {
assert logPrimaryKeyProjection != null;
logPrimaryKey = logPrimaryKeyProjection.apply(record);
}
return logPrimaryKey;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,26 +18,34 @@

package org.apache.paimon.table.sink;

import org.apache.paimon.codegen.CodeGenUtils;
import org.apache.paimon.codegen.Projection;
import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.types.RowType;
import org.apache.paimon.types.RowKind;

/** A {@link PartitionComputer} to compute partition by partition keys. */
public class PartitionComputer {
private final Projection partitionProjection;
/**
* Utility interface to extract partition keys, bucket id, primary keys for file store ({@code
* trimmedPrimaryKey}) and primary keys for external log system ({@code logPrimaryKey}) from the
* given record.
*
* @param <T> type of record
*/
public interface KeyAndBucketExtractor<T> {

public PartitionComputer(TableSchema tableSchema) {
this(tableSchema.logicalRowType(), tableSchema.projection(tableSchema.partitionKeys()));
}
void setRecord(T record);

BinaryRow partition();

int bucket();

BinaryRow trimmedPrimaryKey();

BinaryRow logPrimaryKey();

public PartitionComputer(RowType rowType, int[] partitionKeys) {
this.partitionProjection = CodeGenUtils.newProjection(rowType, partitionKeys);
static int bucketKeyHashCode(BinaryRow bucketKey) {
assert bucketKey.getRowKind() == RowKind.INSERT;
return bucketKey.hashCode();
}

public BinaryRow partition(InternalRow row) {
return this.partitionProjection.apply(row);
static int bucket(int hashcode, int numBuckets) {
return Math.abs(hashcode % numBuckets);
}
}
Loading