|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.flink.cdc.connectors.paimon.sink; |
| 19 | + |
| 20 | +import org.apache.flink.cdc.common.event.CreateTableEvent; |
| 21 | +import org.apache.flink.cdc.common.event.DataChangeEvent; |
| 22 | +import org.apache.flink.cdc.common.event.TableId; |
| 23 | +import org.apache.flink.cdc.common.function.HashFunction; |
| 24 | +import org.apache.flink.cdc.common.schema.Schema; |
| 25 | +import org.apache.flink.cdc.connectors.paimon.sink.v2.PaimonRecordEventSerializer; |
| 26 | + |
| 27 | +import org.apache.paimon.catalog.Catalog; |
| 28 | +import org.apache.paimon.catalog.Identifier; |
| 29 | +import org.apache.paimon.data.GenericRow; |
| 30 | +import org.apache.paimon.flink.FlinkCatalogFactory; |
| 31 | +import org.apache.paimon.flink.sink.RowAssignerChannelComputer; |
| 32 | +import org.apache.paimon.options.Options; |
| 33 | +import org.apache.paimon.table.FileStoreTable; |
| 34 | + |
| 35 | +import java.io.Serializable; |
| 36 | +import java.time.ZoneId; |
| 37 | + |
| 38 | +/** |
| 39 | + * A {@link HashFunction} implementation for {@link PaimonDataSink}. Shuffle {@link DataChangeEvent} |
| 40 | + * by hash of PrimaryKey. |
| 41 | + */ |
| 42 | +public class PaimonHashFunction implements HashFunction<DataChangeEvent>, Serializable { |
| 43 | + |
| 44 | + private static final long serialVersionUID = 1L; |
| 45 | + |
| 46 | + private final PaimonRecordEventSerializer eventSerializer; |
| 47 | + |
| 48 | + private final RowAssignerChannelComputer channelComputer; |
| 49 | + |
| 50 | + public PaimonHashFunction( |
| 51 | + Options options, TableId tableId, Schema schema, ZoneId zoneId, int parallelism) { |
| 52 | + Catalog catalog = FlinkCatalogFactory.createPaimonCatalog(options); |
| 53 | + FileStoreTable table; |
| 54 | + try { |
| 55 | + table = (FileStoreTable) catalog.getTable(Identifier.fromString(tableId.toString())); |
| 56 | + } catch (Catalog.TableNotExistException e) { |
| 57 | + throw new RuntimeException(e); |
| 58 | + } |
| 59 | + eventSerializer = new PaimonRecordEventSerializer(zoneId); |
| 60 | + eventSerializer.serialize(new CreateTableEvent(tableId, schema)); |
| 61 | + channelComputer = new RowAssignerChannelComputer(table.schema(), parallelism); |
| 62 | + channelComputer.setup(parallelism); |
| 63 | + } |
| 64 | + |
| 65 | + @Override |
| 66 | + public int hashcode(DataChangeEvent event) { |
| 67 | + GenericRow genericRow = eventSerializer.serialize(event).getGenericRow(); |
| 68 | + return channelComputer.channel(genericRow); |
| 69 | + } |
| 70 | +} |
0 commit comments