forked from apache/flink-cdc
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-35237][cdc-common] Allow Sink to choose HashFunction in PrePar…
…titionOperator
- Loading branch information
1 parent
634b2d0
commit 237812c
Showing
8 changed files
with
198 additions
and
72 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
93 changes: 93 additions & 0 deletions
93
...dc-common/src/main/java/org/apache/flink/cdc/common/sink/DefaultHashFunctionProvider.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,93 @@ | ||
/* | ||
* 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.flink.cdc.common.sink; | ||
|
||
import org.apache.flink.cdc.common.data.RecordData; | ||
import org.apache.flink.cdc.common.event.DataChangeEvent; | ||
import org.apache.flink.cdc.common.event.OperationType; | ||
import org.apache.flink.cdc.common.event.TableId; | ||
import org.apache.flink.cdc.common.schema.Schema; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.Objects; | ||
import java.util.Optional; | ||
|
||
/** the default implementation of hash function. */ | ||
public class DefaultHashFunctionProvider implements HashFunctionProvider { | ||
private static final long serialVersionUID = 1L; | ||
|
||
@Override | ||
public HashFunction getHashFunction(TableId tableId, Schema schema) { | ||
return new DefaultHashFunction(schema); | ||
} | ||
|
||
static class DefaultHashFunction implements HashFunction { | ||
private final List<RecordData.FieldGetter> primaryKeyGetters; | ||
|
||
public DefaultHashFunction(Schema schema) { | ||
primaryKeyGetters = createFieldGetters(schema); | ||
} | ||
|
||
@Override | ||
public Integer apply(DataChangeEvent event) { | ||
List<Object> objectsToHash = new ArrayList<>(); | ||
// Table ID | ||
TableId tableId = event.tableId(); | ||
Optional.ofNullable(tableId.getNamespace()).ifPresent(objectsToHash::add); | ||
Optional.ofNullable(tableId.getSchemaName()).ifPresent(objectsToHash::add); | ||
objectsToHash.add(tableId.getTableName()); | ||
|
||
// Primary key | ||
RecordData data = | ||
event.op().equals(OperationType.DELETE) ? event.before() : event.after(); | ||
for (RecordData.FieldGetter primaryKeyGetter : primaryKeyGetters) { | ||
objectsToHash.add(primaryKeyGetter.getFieldOrNull(data)); | ||
} | ||
|
||
// Calculate hash | ||
return (Objects.hash(objectsToHash.toArray()) * 31) & 0x7FFFFFFF; | ||
} | ||
|
||
private List<RecordData.FieldGetter> createFieldGetters(Schema schema) { | ||
List<RecordData.FieldGetter> fieldGetters = | ||
new ArrayList<>(schema.primaryKeys().size()); | ||
schema.primaryKeys().stream() | ||
.mapToInt( | ||
pk -> { | ||
int index = schema.getColumnNames().indexOf(pk); | ||
if (index == -1) { | ||
throw new IllegalStateException( | ||
String.format( | ||
"Unable to find column \"%s\" which is defined as primary key", | ||
pk)); | ||
} | ||
return index; | ||
}) | ||
.forEach( | ||
primaryKeyPosition -> | ||
fieldGetters.add( | ||
RecordData.createFieldGetter( | ||
schema.getColumns() | ||
.get(primaryKeyPosition) | ||
.getType(), | ||
primaryKeyPosition))); | ||
return fieldGetters; | ||
} | ||
} | ||
} |
28 changes: 28 additions & 0 deletions
28
flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/HashFunction.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,28 @@ | ||
/* | ||
* 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.flink.cdc.common.sink; | ||
|
||
import org.apache.flink.cdc.common.event.DataChangeEvent; | ||
|
||
import java.util.function.Function; | ||
|
||
/** use for {@code PrePartitionOperator} when calculating hash code of primary key. */ | ||
public interface HashFunction extends Function<DataChangeEvent, Integer> { | ||
@Override | ||
Integer apply(DataChangeEvent event); | ||
} |
43 changes: 43 additions & 0 deletions
43
flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/HashFunctionProvider.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
/* | ||
* 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.flink.cdc.common.sink; | ||
|
||
import org.apache.flink.cdc.common.event.DataChangeEvent; | ||
import org.apache.flink.cdc.common.event.TableId; | ||
import org.apache.flink.cdc.common.schema.Schema; | ||
|
||
import java.io.Serializable; | ||
|
||
/** | ||
* Provide {@link HashFunction} to help {@code PrePartitionOperator} to shuffle {@link | ||
* DataChangeEvent} to designated subtask. This is usually beneficial for load balancing, when | ||
* writing to different partitions/buckets in {@link DataSink}, add custom implementation to further | ||
* improve efficiency. | ||
*/ | ||
public interface HashFunctionProvider extends Serializable { | ||
|
||
/** | ||
* Gets a hash function based on the given table ID and schema, to help {@code | ||
* PrePartitionOperator} to shuffle {@link DataChangeEvent} to designated subtask. | ||
* | ||
* @param tableId table ID | ||
* @param schema flink table schema | ||
* @return hash function based on the given table ID and schema | ||
*/ | ||
HashFunction getHashFunction(TableId tableId, Schema schema); | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters