Skip to content

Commit

Permalink
Flink: make StatisticsOrRecord to be correctly serialized and deser… (
Browse files Browse the repository at this point in the history
#11557)

Co-authored-by: huyuanfeng <[email protected]>
  • Loading branch information
huyuanfeng2018 and huyuanfeng authored Dec 18, 2024
1 parent a6cfc12 commit e3628c1
Show file tree
Hide file tree
Showing 3 changed files with 165 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import org.apache.iceberg.flink.sink.shuffle.DataStatisticsOperatorFactory;
import org.apache.iceberg.flink.sink.shuffle.RangePartitioner;
import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord;
import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecordTypeInformation;
import org.apache.iceberg.flink.sink.shuffle.StatisticsType;
import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
Expand Down Expand Up @@ -644,12 +645,14 @@ private DataStream<RowData> distributeDataStream(
}

LOG.info("Range distribute rows by sort order: {}", sortOrder);
StatisticsOrRecordTypeInformation statisticsOrRecordTypeInformation =
new StatisticsOrRecordTypeInformation(flinkRowType, iSchema, sortOrder);
StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType();
SingleOutputStreamOperator<StatisticsOrRecord> shuffleStream =
input
.transform(
operatorName("range-shuffle"),
TypeInformation.of(StatisticsOrRecord.class),
statisticsOrRecordTypeInformation,
new DataStatisticsOperatorFactory(
iSchema,
sortOrder,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* 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.shuffle;

import java.util.Objects;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.serialization.SerializerConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.Schema;
import org.apache.iceberg.SortOrder;
import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;

public class StatisticsOrRecordTypeInformation extends TypeInformation<StatisticsOrRecord> {

private final TypeInformation<RowData> rowTypeInformation;
private final SortOrder sortOrder;
private final GlobalStatisticsSerializer globalStatisticsSerializer;

public StatisticsOrRecordTypeInformation(
RowType flinkRowType, Schema schema, SortOrder sortOrder) {
this.sortOrder = sortOrder;
this.rowTypeInformation = FlinkCompatibilityUtil.toTypeInfo(flinkRowType);
this.globalStatisticsSerializer =
new GlobalStatisticsSerializer(new SortKeySerializer(schema, sortOrder));
}

@Override
public boolean isBasicType() {
return false;
}

@Override
public boolean isTupleType() {
return false;
}

@Override
public int getArity() {
return 1;
}

@Override
public int getTotalFields() {
return 1;
}

@Override
public Class<StatisticsOrRecord> getTypeClass() {
return StatisticsOrRecord.class;
}

@Override
public boolean isKeyType() {
return false;
}

@Override
public TypeSerializer<StatisticsOrRecord> createSerializer(SerializerConfig config) {
TypeSerializer<RowData> recordSerializer = rowTypeInformation.createSerializer(config);
return new StatisticsOrRecordSerializer(globalStatisticsSerializer, recordSerializer);
}

@Override
public TypeSerializer<StatisticsOrRecord> createSerializer(ExecutionConfig config) {
return createSerializer(config.getSerializerConfig());
}

@Override
public String toString() {
return "StatisticsOrRecord";
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
} else if (o != null && this.getClass() == o.getClass()) {
StatisticsOrRecordTypeInformation that = (StatisticsOrRecordTypeInformation) o;
return that.sortOrder.equals(sortOrder)
&& that.rowTypeInformation.equals(rowTypeInformation)
&& that.globalStatisticsSerializer.equals(globalStatisticsSerializer);
} else {
return false;
}
}

@Override
public int hashCode() {
return Objects.hash(rowTypeInformation, sortOrder, globalStatisticsSerializer);
}

@Override
public boolean canEqual(Object obj) {
return obj instanceof StatisticsOrRecordTypeInformation;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.shuffle;

import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.Schema;
import org.apache.iceberg.SortOrder;
import org.apache.iceberg.flink.FlinkSchemaUtil;
import org.apache.iceberg.types.Types;

public class TestStatisticsOrRecordTypeInformation
extends TypeInformationTestBase<StatisticsOrRecordTypeInformation> {
private static final Schema SCHEMA =
new Schema(
Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()),
Types.NestedField.optional(2, "uuid", Types.UUIDType.get()),
Types.NestedField.optional(3, "data", Types.StringType.get()));
private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA);
private static final SortOrder SORT_ORDER1 = SortOrder.builderFor(SCHEMA).asc("ts").build();
private static final SortOrder SORT_ORDER2 = SortOrder.builderFor(SCHEMA).asc("data").build();

@Override
protected StatisticsOrRecordTypeInformation[] getTestData() {
return new StatisticsOrRecordTypeInformation[] {
new StatisticsOrRecordTypeInformation(ROW_TYPE, SCHEMA, SORT_ORDER1),
new StatisticsOrRecordTypeInformation(ROW_TYPE, SCHEMA, SORT_ORDER2),
};
}
}

0 comments on commit e3628c1

Please sign in to comment.