You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by cz...@apache.org on 2022/11/04 03:10:07 UTC

[flink-table-store] branch master updated: [FLINK-29848] Port BinaryRowDataUtil to Flink Table Store

This is an automated email from the ASF dual-hosted git repository.

czweng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink-table-store.git


The following commit(s) were added to refs/heads/master by this push:
     new 18c01207 [FLINK-29848] Port BinaryRowDataUtil to Flink Table Store
18c01207 is described below

commit 18c01207839a6b2eade3a010538a4aa1610a2eb6
Author: Jane Chan <55...@users.noreply.github.com>
AuthorDate: Fri Nov 4 11:10:02 2022 +0800

    [FLINK-29848] Port BinaryRowDataUtil to Flink Table Store
    
    This closes #348.
---
 .../file/mergetree/MergeTreeBenchmark.java         |  2 +-
 .../flink/table/store/utils/BinaryRowDataUtil.java | 67 ++++++++++++++++++++++
 .../flink/table/store/codegen/CodeGenUtils.java    |  2 +-
 .../flink/table/store/file/io/DataFileMeta.java    |  2 +-
 .../store/file/append/AppendOnlyWriterTest.java    |  2 +-
 .../store/file/io/KeyValueFileReadWriteTest.java   |  2 +-
 .../table/store/file/mergetree/MergeTreeTest.java  |  2 +-
 .../store/mapred/TableStoreRecordReaderTest.java   |  2 +-
 .../flink/table/store/kafka/KafkaLogTestUtils.java |  2 +-
 9 files changed, 75 insertions(+), 8 deletions(-)

diff --git a/flink-table-store-benchmark/flink-table-store-micro-benchmarks/src/main/java/org/apache/flink/table/store/benchmark/file/mergetree/MergeTreeBenchmark.java b/flink-table-store-benchmark/flink-table-store-micro-benchmarks/src/main/java/org/apache/flink/table/store/benchmark/file/mergetree/MergeTreeBenchmark.java
index 5c6f3be5..33042cc7 100644
--- a/flink-table-store-benchmark/flink-table-store-micro-benchmarks/src/main/java/org/apache/flink/table/store/benchmark/file/mergetree/MergeTreeBenchmark.java
+++ b/flink-table-store-benchmark/flink-table-store-micro-benchmarks/src/main/java/org/apache/flink/table/store/benchmark/file/mergetree/MergeTreeBenchmark.java
@@ -22,7 +22,6 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.binary.BinaryRowDataUtil;
 import org.apache.flink.table.store.CoreOptions;
 import org.apache.flink.table.store.benchmark.config.ConfigUtil;
 import org.apache.flink.table.store.benchmark.config.FileBenchmarkOptions;
@@ -48,6 +47,7 @@ import org.apache.flink.table.store.file.utils.FileStorePathFactory;
 import org.apache.flink.table.store.file.utils.RecordReaderIterator;
 import org.apache.flink.table.store.file.utils.RecordWriter;
 import org.apache.flink.table.store.format.FileFormat;
+import org.apache.flink.table.store.utils.BinaryRowDataUtil;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.types.RowKind;
diff --git a/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/BinaryRowDataUtil.java b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/BinaryRowDataUtil.java
new file mode 100644
index 00000000..bf014a04
--- /dev/null
+++ b/flink-table-store-common/src/main/java/org/apache/flink/table/store/utils/BinaryRowDataUtil.java
@@ -0,0 +1,67 @@
+/*
+ * 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.table.store.utils;
+
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.core.memory.MemoryUtils;
+import org.apache.flink.table.data.binary.BinaryRowData;
+
+/**
+ * Utilities for {@link BinaryRowData}. Many of the methods in this class are used in code
+ * generation.
+ *
+ * <p>This is directly copied from {@link org.apache.flink.table.data.binary.BinaryRowDataUtil}.
+ */
+public class BinaryRowDataUtil {
+
+    public static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+    public static final int BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+    public static final BinaryRowData EMPTY_ROW = new BinaryRowData(0);
+
+    static {
+        int size = EMPTY_ROW.getFixedLengthPartSize();
+        byte[] bytes = new byte[size];
+        EMPTY_ROW.pointTo(MemorySegmentFactory.wrap(bytes), 0, size);
+    }
+
+    public static boolean byteArrayEquals(byte[] left, byte[] right, int length) {
+        return byteArrayEquals(left, BYTE_ARRAY_BASE_OFFSET, right, BYTE_ARRAY_BASE_OFFSET, length);
+    }
+
+    public static boolean byteArrayEquals(
+            Object left, long leftOffset, Object right, long rightOffset, int length) {
+        int i = 0;
+
+        while (i <= length - 8) {
+            if (UNSAFE.getLong(left, leftOffset + i) != UNSAFE.getLong(right, rightOffset + i)) {
+                return false;
+            }
+            i += 8;
+        }
+
+        while (i < length) {
+            if (UNSAFE.getByte(left, leftOffset + i) != UNSAFE.getByte(right, rightOffset + i)) {
+                return false;
+            }
+            i += 1;
+        }
+        return true;
+    }
+}
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/codegen/CodeGenUtils.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/codegen/CodeGenUtils.java
index 23622f5d..da5e2e4e 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/codegen/CodeGenUtils.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/codegen/CodeGenUtils.java
@@ -20,11 +20,11 @@ package org.apache.flink.table.store.codegen;
 
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.data.binary.BinaryRowDataUtil;
 import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
 import org.apache.flink.table.runtime.generated.Projection;
 import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.store.utils.BinaryRowDataUtil;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 
diff --git a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/DataFileMeta.java b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/DataFileMeta.java
index 570faa35..ce46b16f 100644
--- a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/DataFileMeta.java
+++ b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/io/DataFileMeta.java
@@ -32,9 +32,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
-import static org.apache.flink.table.data.binary.BinaryRowDataUtil.EMPTY_ROW;
 import static org.apache.flink.table.store.file.utils.SerializationUtils.newBytesType;
 import static org.apache.flink.table.store.file.utils.SerializationUtils.newStringType;
+import static org.apache.flink.table.store.utils.BinaryRowDataUtil.EMPTY_ROW;
 import static org.apache.flink.util.Preconditions.checkArgument;
 
 /** Metadata of a data file. */
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/append/AppendOnlyWriterTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/append/AppendOnlyWriterTest.java
index 7accc95c..e37bc02c 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/append/AppendOnlyWriterTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/append/AppendOnlyWriterTest.java
@@ -25,7 +25,6 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.StringData;
-import org.apache.flink.table.data.binary.BinaryRowDataUtil;
 import org.apache.flink.table.store.CoreOptions;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.file.io.DataFilePathFactory;
@@ -33,6 +32,7 @@ import org.apache.flink.table.store.file.stats.FieldStatsArraySerializer;
 import org.apache.flink.table.store.file.utils.RecordWriter;
 import org.apache.flink.table.store.format.FieldStats;
 import org.apache.flink.table.store.format.FileFormat;
+import org.apache.flink.table.store.utils.BinaryRowDataUtil;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/io/KeyValueFileReadWriteTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/io/KeyValueFileReadWriteTest.java
index a25164e5..5f8e639f 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/io/KeyValueFileReadWriteTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/io/KeyValueFileReadWriteTest.java
@@ -23,7 +23,6 @@ import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.binary.BinaryRowDataUtil;
 import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
 import org.apache.flink.table.store.file.KeyValue;
 import org.apache.flink.table.store.file.KeyValueSerializerTest;
@@ -35,6 +34,7 @@ import org.apache.flink.table.store.file.stats.StatsTestUtils;
 import org.apache.flink.table.store.file.utils.FailingAtomicRenameFileSystem;
 import org.apache.flink.table.store.file.utils.FileStorePathFactory;
 import org.apache.flink.table.store.file.utils.RecordReaderIterator;
+import org.apache.flink.table.store.utils.BinaryRowDataUtil;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.LogicalType;
diff --git a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/MergeTreeTest.java b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/MergeTreeTest.java
index b846f565..9b7988bd 100644
--- a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/MergeTreeTest.java
+++ b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/mergetree/MergeTreeTest.java
@@ -24,7 +24,6 @@ import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.binary.BinaryRowDataUtil;
 import org.apache.flink.table.store.CoreOptions;
 import org.apache.flink.table.store.CoreOptions.ChangelogProducer;
 import org.apache.flink.table.store.file.KeyValue;
@@ -47,6 +46,7 @@ import org.apache.flink.table.store.file.utils.RecordReader;
 import org.apache.flink.table.store.file.utils.RecordReaderIterator;
 import org.apache.flink.table.store.file.utils.RecordWriter;
 import org.apache.flink.table.store.format.FileFormat;
+import org.apache.flink.table.store.utils.BinaryRowDataUtil;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.types.RowKind;
diff --git a/flink-table-store-hive/flink-table-store-hive-connector/src/test/java/org/apache/flink/table/store/mapred/TableStoreRecordReaderTest.java b/flink-table-store-hive/flink-table-store-hive-connector/src/test/java/org/apache/flink/table/store/mapred/TableStoreRecordReaderTest.java
index 79c57848..3b1b56bc 100644
--- a/flink-table-store-hive/flink-table-store-hive-connector/src/test/java/org/apache/flink/table/store/mapred/TableStoreRecordReaderTest.java
+++ b/flink-table-store-hive/flink-table-store-hive-connector/src/test/java/org/apache/flink/table/store/mapred/TableStoreRecordReaderTest.java
@@ -24,7 +24,6 @@ import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.data.StringData;
 import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.data.binary.BinaryRowDataUtil;
 import org.apache.flink.table.store.CoreOptions;
 import org.apache.flink.table.store.FileStoreTestUtils;
 import org.apache.flink.table.store.RowDataContainer;
@@ -32,6 +31,7 @@ import org.apache.flink.table.store.table.FileStoreTable;
 import org.apache.flink.table.store.table.sink.TableCommit;
 import org.apache.flink.table.store.table.sink.TableWrite;
 import org.apache.flink.table.store.table.source.DataSplit;
+import org.apache.flink.table.store.utils.BinaryRowDataUtil;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.types.RowKind;
diff --git a/flink-table-store-kafka/src/test/java/org/apache/flink/table/store/kafka/KafkaLogTestUtils.java b/flink-table-store-kafka/src/test/java/org/apache/flink/table/store/kafka/KafkaLogTestUtils.java
index 70b45314..c7ddd5ed 100644
--- a/flink-table-store-kafka/src/test/java/org/apache/flink/table/store/kafka/KafkaLogTestUtils.java
+++ b/flink-table-store-kafka/src/test/java/org/apache/flink/table/store/kafka/KafkaLogTestUtils.java
@@ -55,12 +55,12 @@ import java.util.UUID;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
-import static org.apache.flink.table.data.binary.BinaryRowDataUtil.EMPTY_ROW;
 import static org.apache.flink.table.store.CoreOptions.LOG_CHANGELOG_MODE;
 import static org.apache.flink.table.store.CoreOptions.LOG_CONSISTENCY;
 import static org.apache.flink.table.store.file.mergetree.compact.MergeTreeCompactManagerTest.row;
 import static org.apache.flink.table.store.kafka.KafkaLogOptions.BOOTSTRAP_SERVERS;
 import static org.apache.flink.table.store.kafka.KafkaLogOptions.TOPIC;
+import static org.apache.flink.table.store.utils.BinaryRowDataUtil.EMPTY_ROW;
 
 /** Utils for the test of {@link KafkaLogStoreFactory}. */
 public class KafkaLogTestUtils {