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
3 changes: 3 additions & 0 deletions flink/v1.19/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,9 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") {

implementation libs.datasketches

// for caching in DynamicSink
implementation libs.caffeine

testImplementation libs.flink119.connector.test.utils
testImplementation libs.flink119.core
testImplementation libs.flink119.runtime
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
/*
* 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.dynamic;

import java.util.List;
import javax.annotation.Nullable;
import org.apache.flink.table.data.RowData;
import org.apache.iceberg.DistributionMode;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.catalog.TableIdentifier;

/** A DynamicRecord contains RowData alongside with the Iceberg table metadata. */
public class DynamicRecord {

private TableIdentifier tableIdentifier;
private String branch;
private Schema schema;
private RowData rowData;
private PartitionSpec partitionSpec;
private DistributionMode distributionMode;
private int writeParallelism;
private boolean upsertMode;
@Nullable private List<String> equalityFields;

public DynamicRecord(
TableIdentifier tableIdentifier,
String branch,
Schema schema,
RowData rowData,
PartitionSpec partitionSpec,
DistributionMode distributionMode,
int writeParallelism) {
this.tableIdentifier = tableIdentifier;
this.branch = branch;
this.schema = schema;
this.partitionSpec = partitionSpec;
this.rowData = rowData;
this.distributionMode = distributionMode;
this.writeParallelism = writeParallelism;
}

public TableIdentifier tableIdentifier() {
return tableIdentifier;
}

public void setTableIdentifier(TableIdentifier tableIdentifier) {
this.tableIdentifier = tableIdentifier;
}

public String branch() {
return branch;
}

public void setBranch(String branch) {
this.branch = branch;
}

public Schema schema() {
return schema;
}

public void setSchema(Schema schema) {
this.schema = schema;
}

public PartitionSpec spec() {
return partitionSpec;
}

public void setPartitionSpec(PartitionSpec partitionSpec) {
this.partitionSpec = partitionSpec;
}

public RowData rowData() {
return rowData;
}

public void setRowData(RowData rowData) {
this.rowData = rowData;
}

public DistributionMode distributionMode() {
return distributionMode;
}

public void setDistributionMode(DistributionMode distributionMode) {
this.distributionMode = distributionMode;
}

public int writeParallelism() {
return writeParallelism;
}

public void writeParallelism(int parallelism) {
this.writeParallelism = parallelism;
}

public boolean upsertMode() {
return upsertMode;
}

public void setUpsertMode(boolean upsertMode) {
this.upsertMode = upsertMode;
}

public List<String> equalityFields() {
return equalityFields;
}

public void setEqualityFields(List<String> equalityFields) {
this.equalityFields = equalityFields;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,166 @@
/*
* 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.dynamic;

import java.util.List;
import java.util.Objects;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.flink.FlinkSchemaUtil;

@Internal
class DynamicRecordInternal {

private String tableName;
private String branch;
private Schema schema;
private PartitionSpec spec;
private int writerKey;
private RowData rowData;
private boolean upsertMode;
private List<Integer> equalityFieldIds;

// Required for serialization instantiation
DynamicRecordInternal() {}

DynamicRecordInternal(
String tableName,
String branch,
Schema schema,
RowData rowData,
PartitionSpec spec,
int writerKey,
boolean upsertMode,
List<Integer> equalityFieldsIds) {
this.tableName = tableName;
this.branch = branch;
this.schema = schema;
this.spec = spec;
this.writerKey = writerKey;
this.rowData = rowData;
this.upsertMode = upsertMode;
this.equalityFieldIds = equalityFieldsIds;
}

public String tableName() {
return tableName;
}

public void setTableName(String tableName) {
this.tableName = tableName;
}

public String branch() {
return branch;
}

public void setBranch(String branch) {
this.branch = branch;
}

public Schema schema() {
return schema;
}

public void setSchema(Schema schema) {
this.schema = schema;
}

public RowData rowData() {
return rowData;
}

public void setRowData(RowData rowData) {
this.rowData = rowData;
}

public PartitionSpec spec() {
return spec;
}

public void setSpec(PartitionSpec spec) {
this.spec = spec;
}

public int writerKey() {
return writerKey;
}

public void setWriterKey(int writerKey) {
this.writerKey = writerKey;
}

public boolean upsertMode() {
return upsertMode;
}

public void setUpsertMode(boolean upsertMode) {
this.upsertMode = upsertMode;
}

public List<Integer> equalityFields() {
return equalityFieldIds;
}

public void setEqualityFieldIds(List<Integer> equalityFieldIds) {
this.equalityFieldIds = equalityFieldIds;
}

@Override
public int hashCode() {
return Objects.hash(
tableName, branch, schema, spec, writerKey, rowData, upsertMode, equalityFieldIds);
}

@Override
public boolean equals(Object other) {
if (this == other) {
return true;
}

if (other == null || getClass() != other.getClass()) {
return false;
}

DynamicRecordInternal that = (DynamicRecordInternal) other;
boolean tableFieldsMatch =
Objects.equals(tableName, that.tableName)
&& Objects.equals(branch, that.branch)
&& schema.schemaId() == that.schema.schemaId()
&& Objects.equals(spec, that.spec)
&& writerKey == that.writerKey
&& upsertMode == that.upsertMode
&& Objects.equals(equalityFieldIds, that.equalityFieldIds);
if (!tableFieldsMatch) {
return false;
}

if (rowData.getClass().equals(that.rowData.getClass())) {
return Objects.equals(rowData, that.rowData);
} else {
RowDataSerializer rowDataSerializer = new RowDataSerializer(FlinkSchemaUtil.convert(schema));
return rowDataSerializer
.toBinaryRow(rowData)
.equals(rowDataSerializer.toBinaryRow(that.rowData));
}
}
}
Loading