You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2021/12/07 16:20:27 UTC

[iceberg] branch master updated: Flink: Support Flink 1.14.0 (#3434)

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

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 2d4b0dd  Flink: Support Flink 1.14.0 (#3434)
2d4b0dd is described below

commit 2d4b0ddc76fd47aa27ca4972d4f3a6f256921c58
Author: openinx <op...@gmail.com>
AuthorDate: Wed Dec 8 00:20:16 2021 +0800

    Flink: Support Flink 1.14.0 (#3434)
---
 .github/workflows/flink-ci.yml                     |   2 +-
 .github/workflows/java-ci.yml                      |   2 +-
 flink/build.gradle                                 |   6 +-
 flink/v1.14/build.gradle                           | 175 +++++
 flink/v1.14/flink-runtime/LICENSE                  | 484 ++++++++++++
 flink/v1.14/flink-runtime/NOTICE                   |  91 +++
 .../org/apache/iceberg/flink/CatalogLoader.java    | 158 ++++
 .../org/apache/iceberg/flink/FlinkCatalog.java     | 724 ++++++++++++++++++
 .../apache/iceberg/flink/FlinkCatalogFactory.java  | 162 ++++
 .../apache/iceberg/flink/FlinkConfigOptions.java   |  43 ++
 .../iceberg/flink/FlinkDynamicTableFactory.java    | 201 +++++
 .../org/apache/iceberg/flink/FlinkFilters.java     | 252 +++++++
 .../org/apache/iceberg/flink/FlinkFixupTypes.java  |  51 ++
 .../org/apache/iceberg/flink/FlinkSchemaUtil.java  | 175 +++++
 .../org/apache/iceberg/flink/FlinkTypeToType.java  | 199 +++++
 .../org/apache/iceberg/flink/FlinkTypeVisitor.java |  81 ++
 .../org/apache/iceberg/flink/IcebergTableSink.java |  98 +++
 .../apache/iceberg/flink/IcebergTableSource.java   | 182 +++++
 .../org/apache/iceberg/flink/RowDataWrapper.java   | 138 ++++
 .../java/org/apache/iceberg/flink/TableLoader.java | 131 ++++
 .../org/apache/iceberg/flink/TypeToFlinkType.java  | 134 ++++
 .../org/apache/iceberg/flink/actions/Actions.java  |  53 ++
 .../flink/actions/RewriteDataFilesAction.java      |  72 ++
 .../flink/data/AvroWithFlinkSchemaVisitor.java     |  73 ++
 .../apache/iceberg/flink/data/FlinkAvroReader.java | 167 +++++
 .../apache/iceberg/flink/data/FlinkAvroWriter.java | 156 ++++
 .../apache/iceberg/flink/data/FlinkOrcReader.java  | 125 ++++
 .../apache/iceberg/flink/data/FlinkOrcReaders.java | 272 +++++++
 .../apache/iceberg/flink/data/FlinkOrcWriter.java  | 151 ++++
 .../apache/iceberg/flink/data/FlinkOrcWriters.java | 294 ++++++++
 .../iceberg/flink/data/FlinkParquetReaders.java    | 799 ++++++++++++++++++++
 .../iceberg/flink/data/FlinkParquetWriters.java    | 456 ++++++++++++
 .../iceberg/flink/data/FlinkSchemaVisitor.java     | 164 ++++
 .../iceberg/flink/data/FlinkValueReaders.java      | 313 ++++++++
 .../iceberg/flink/data/FlinkValueWriters.java      | 241 ++++++
 .../flink/data/ParquetWithFlinkSchemaVisitor.java  | 199 +++++
 .../iceberg/flink/data/RowDataProjection.java      | 239 ++++++
 .../org/apache/iceberg/flink/data/RowDataUtil.java | 103 +++
 .../iceberg/flink/sink/BaseDeltaTaskWriter.java    | 107 +++
 .../apache/iceberg/flink/sink/DeltaManifests.java  |  71 ++
 .../flink/sink/DeltaManifestsSerializer.java       | 122 +++
 .../iceberg/flink/sink/FlinkAppenderFactory.java   | 251 +++++++
 .../iceberg/flink/sink/FlinkFileWriterFactory.java | 259 +++++++
 .../iceberg/flink/sink/FlinkManifestUtil.java      | 119 +++
 .../org/apache/iceberg/flink/sink/FlinkSink.java   | 478 ++++++++++++
 .../iceberg/flink/sink/IcebergFilesCommitter.java  | 383 ++++++++++
 .../iceberg/flink/sink/IcebergStreamWriter.java    | 103 +++
 .../flink/sink/ManifestOutputFileFactory.java      |  78 ++
 .../iceberg/flink/sink/PartitionKeySelector.java   |  64 ++
 .../iceberg/flink/sink/PartitionedDeltaWriter.java |  87 +++
 .../flink/sink/RowDataTaskWriterFactory.java       | 128 ++++
 .../iceberg/flink/sink/TaskWriterFactory.java}     |  32 +-
 .../flink/sink/UnpartitionedDeltaWriter.java       |  60 ++
 .../apache/iceberg/flink/source/DataIterator.java  |  93 +++
 .../iceberg/flink/source/FileScanTaskReader.java}  |  22 +-
 .../iceberg/flink/source/FlinkInputFormat.java     | 119 +++
 .../iceberg/flink/source/FlinkInputSplit.java      |  56 ++
 .../apache/iceberg/flink/source/FlinkSource.java   | 255 +++++++
 .../iceberg/flink/source/FlinkSplitGenerator.java  |  92 +++
 .../flink/source/RowDataFileScanTaskReader.java    | 196 +++++
 .../iceberg/flink/source/RowDataRewriter.java      | 155 ++++
 .../apache/iceberg/flink/source/ScanContext.java   | 319 ++++++++
 .../flink/source/StreamingMonitorFunction.java     | 177 +++++
 .../flink/source/StreamingReaderOperator.java      | 232 ++++++
 .../flink/util/FlinkCompatibilityUtil.java}        |  31 +-
 .../org.apache.flink.table.factories.Factory       |  13 +-
 .../org.apache.flink.table.factories.TableFactory  |  13 +-
 .../apache/iceberg/flink/FlinkCatalogTestBase.java | 149 ++++
 .../org/apache/iceberg/flink/FlinkTestBase.java    | 106 +++
 .../apache/iceberg/flink/MiniClusterResource.java  |  57 ++
 .../org/apache/iceberg/flink/RowDataConverter.java | 148 ++++
 .../org/apache/iceberg/flink/SimpleDataUtil.java   | 310 ++++++++
 .../iceberg/flink/TestCatalogTableLoader.java      | 127 ++++
 .../apache/iceberg/flink/TestChangeLogTable.java   | 317 ++++++++
 .../iceberg/flink/TestDataFileSerialization.java   | 201 +++++
 .../org/apache/iceberg/flink/TestFixtures.java     |  52 ++
 .../iceberg/flink/TestFlinkCatalogDatabase.java    | 276 +++++++
 .../iceberg/flink/TestFlinkCatalogFactory.java     | 119 +++
 .../iceberg/flink/TestFlinkCatalogTable.java       | 401 ++++++++++
 .../flink/TestFlinkCatalogTablePartitions.java     | 115 +++
 .../org/apache/iceberg/flink/TestFlinkFilters.java | 405 ++++++++++
 .../apache/iceberg/flink/TestFlinkHiveCatalog.java | 102 +++
 .../apache/iceberg/flink/TestFlinkSchemaUtil.java  | 328 ++++++++
 .../apache/iceberg/flink/TestFlinkTableSink.java   | 294 ++++++++
 .../apache/iceberg/flink/TestFlinkTableSource.java | 616 +++++++++++++++
 .../java/org/apache/iceberg/flink/TestHelpers.java | 356 +++++++++
 .../apache/iceberg/flink/TestIcebergConnector.java | 358 +++++++++
 .../flink/TestManifestFileSerialization.java       | 170 +++++
 .../apache/iceberg/flink/TestRowDataWrapper.java   |  89 +++
 .../org/apache/iceberg/flink/TestTableLoader.java} |  37 +-
 .../flink/actions/TestRewriteDataFilesAction.java  | 386 ++++++++++
 .../apache/iceberg/flink/data/RandomRowData.java}  |  26 +-
 .../flink/data/TestFlinkAvroReaderWriter.java      | 101 +++
 .../flink/data/TestFlinkOrcReaderWriter.java       | 101 +++
 .../iceberg/flink/data/TestFlinkParquetReader.java |  75 ++
 .../iceberg/flink/data/TestFlinkParquetWriter.java |  89 +++
 .../iceberg/flink/data/TestRowDataProjection.java  | 332 +++++++++
 .../iceberg/flink/data/TestRowProjection.java      | 572 ++++++++++++++
 .../iceberg/flink/sink/TestDeltaTaskWriter.java    | 341 +++++++++
 .../flink/sink/TestFlinkAppenderFactory.java       |  65 ++
 .../flink/sink/TestFlinkFileWriterFactory.java     |  70 ++
 .../iceberg/flink/sink/TestFlinkIcebergSink.java   | 319 ++++++++
 .../iceberg/flink/sink/TestFlinkIcebergSinkV2.java | 465 ++++++++++++
 .../iceberg/flink/sink/TestFlinkManifest.java      | 274 +++++++
 .../flink/sink/TestFlinkPartitioningWriters.java   |  70 ++
 .../flink/sink/TestFlinkPositionDeltaWriters.java  |  70 ++
 .../flink/sink/TestFlinkRollingFileWriters.java    |  55 ++
 .../iceberg/flink/sink/TestFlinkWriterMetrics.java |  52 ++
 .../flink/sink/TestIcebergFilesCommitter.java      | 829 +++++++++++++++++++++
 .../flink/sink/TestIcebergStreamWriter.java        | 352 +++++++++
 .../flink/sink/TestRowDataPartitionKey.java        | 241 ++++++
 .../apache/iceberg/flink/sink/TestTaskWriters.java | 246 ++++++
 .../iceberg/flink/source/BoundedTableFactory.java  | 151 ++++
 .../iceberg/flink/source/BoundedTestSource.java    |  95 +++
 .../flink/source/ChangeLogTableTestBase.java       |  93 +++
 .../flink/source/TestBoundedTableFactory.java      |  81 ++
 .../iceberg/flink/source/TestFlinkInputFormat.java | 133 ++++
 .../source/TestFlinkInputFormatReaderDeletes.java  |  68 ++
 .../flink/source/TestFlinkMergingMetrics.java      |  54 ++
 .../flink/source/TestFlinkReaderDeletesBase.java   | 110 +++
 .../apache/iceberg/flink/source/TestFlinkScan.java | 320 ++++++++
 .../iceberg/flink/source/TestFlinkScanSql.java     | 194 +++++
 .../iceberg/flink/source/TestFlinkSource.java      |  78 ++
 .../iceberg/flink/source/TestStreamScanSql.java    | 245 ++++++
 .../flink/source/TestStreamingMonitorFunction.java | 301 ++++++++
 .../flink/source/TestStreamingReaderOperator.java  | 284 +++++++
 .../org.apache.flink.table.factories.Factory       |  13 +-
 gradle.properties                                  |   4 +-
 settings.gradle                                    |   9 +
 129 files changed, 23905 insertions(+), 73 deletions(-)

diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml
index ce6e9b3..840ec33 100644
--- a/.github/workflows/flink-ci.yml
+++ b/.github/workflows/flink-ci.yml
@@ -59,7 +59,7 @@ jobs:
     strategy:
       matrix:
         jvm: [8, 11]
-        flink: ['1.12', '1.13']
+        flink: ['1.12', '1.13', '1.14']
     env:
       SPARK_LOCAL_IP: localhost
     steps:
diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml
index 6e47eaa..7f381d5 100644
--- a/.github/workflows/java-ci.yml
+++ b/.github/workflows/java-ci.yml
@@ -77,7 +77,7 @@ jobs:
     - uses: actions/setup-java@v1
       with:
         java-version: 8
-    - run: ./gradlew -DflinkVersions=1.12,1.13 -DsparkVersions=2.4,3.0,3.1,3.2 -DhiveVersions=2,3 build -x test -x javadoc -x integrationTest
+    - run: ./gradlew -DflinkVersions=1.12,1.13,1.14 -DsparkVersions=2.4,3.0,3.1,3.2 -DhiveVersions=2,3 build -x test -x javadoc -x integrationTest
 
   build-javadoc:
     runs-on: ubuntu-latest
diff --git a/flink/build.gradle b/flink/build.gradle
index 41b9e63..0934a50 100644
--- a/flink/build.gradle
+++ b/flink/build.gradle
@@ -25,4 +25,8 @@ if (flinkVersions.contains("1.12")) {
 
 if (flinkVersions.contains("1.13")) {
   apply from: file("$projectDir/v1.13/build.gradle")
-}
\ No newline at end of file
+}
+
+if (flinkVersions.contains("1.14")) {
+  apply from: file("$projectDir/v1.14/build.gradle")
+}
diff --git a/flink/v1.14/build.gradle b/flink/v1.14/build.gradle
new file mode 100644
index 0000000..2b2df13
--- /dev/null
+++ b/flink/v1.14/build.gradle
@@ -0,0 +1,175 @@
+/*
+ * 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.
+ */
+
+def flinkProjects = [
+  project(':iceberg-flink:iceberg-flink-1.14'),
+  project(':iceberg-flink:iceberg-flink-1.14-runtime')
+]
+
+configure(flinkProjects) {
+  project.ext {
+    flinkVersion = '1.14.0'
+  }
+}
+
+project(':iceberg-flink:iceberg-flink-1.14') {
+
+  dependencies {
+    implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
+    api project(':iceberg-api')
+    implementation project(':iceberg-common')
+    implementation project(':iceberg-core')
+    api project(':iceberg-data')
+    implementation project(':iceberg-orc')
+    implementation project(':iceberg-parquet')
+    implementation project(':iceberg-hive-metastore')
+
+    compileOnly "org.apache.flink:flink-streaming-java_2.12:${flinkVersion}"
+    compileOnly "org.apache.flink:flink-streaming-java_2.12:${flinkVersion}:tests"
+    compileOnly "org.apache.flink:flink-table-api-java-bridge_2.12:${flinkVersion}"
+    compileOnly "org.apache.flink:flink-table-planner_2.12:${flinkVersion}"
+    compileOnly "org.apache.hadoop:hadoop-hdfs"
+    compileOnly "org.apache.hadoop:hadoop-common"
+    compileOnly("org.apache.hadoop:hadoop-minicluster") {
+      exclude group: 'org.apache.avro', module: 'avro'
+    }
+
+    implementation("org.apache.parquet:parquet-avro") {
+      exclude group: 'org.apache.avro', module: 'avro'
+      // already shaded by Parquet
+      exclude group: 'it.unimi.dsi'
+      exclude group: 'org.codehaus.jackson'
+    }
+
+    compileOnly "org.apache.avro:avro"
+
+    implementation("org.apache.orc:orc-core::nohive") {
+      exclude group: 'org.apache.hadoop'
+      exclude group: 'commons-lang'
+      // These artifacts are shaded and included in the orc-core fat jar
+      exclude group: 'com.google.protobuf', module: 'protobuf-java'
+      exclude group: 'org.apache.hive', module: 'hive-storage-api'
+    }
+
+    testImplementation "org.apache.flink:flink-core:${flinkVersion}"
+    testImplementation "org.apache.flink:flink-runtime:${flinkVersion}"
+    testImplementation ("org.apache.flink:flink-test-utils-junit:${flinkVersion}") {
+      exclude group: 'junit'
+    }
+    testImplementation("org.apache.flink:flink-test-utils_2.12:${flinkVersion}") {
+      exclude group: "org.apache.curator", module: 'curator-test'
+      exclude group: 'junit'
+    }
+
+    testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
+    testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts')
+    testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts')
+    testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts')
+
+    // By default, hive-exec is a fat/uber jar and it exports a guava library
+    // that's really old. We use the core classifier to be able to override our guava
+    // version. Luckily, hive-exec seems to work okay so far with this version of guava
+    // See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for more context.
+    testImplementation("org.apache.hive:hive-exec::core") {
+      exclude group: 'org.apache.avro', module: 'avro'
+      exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+      exclude group: 'org.pentaho' // missing dependency
+      exclude group: 'org.apache.hive', module: 'hive-llap-tez'
+      exclude group: 'org.apache.logging.log4j'
+      exclude group: 'com.google.protobuf', module: 'protobuf-java'
+      exclude group: 'org.apache.calcite'
+      exclude group: 'org.apache.calcite.avatica'
+      exclude group: 'com.google.code.findbugs', module: 'jsr305'
+    }
+
+    testImplementation("org.apache.hive:hive-metastore") {
+      exclude group: 'org.apache.avro', module: 'avro'
+      exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+      exclude group: 'org.pentaho' // missing dependency
+      exclude group: 'org.apache.hbase'
+      exclude group: 'org.apache.logging.log4j'
+      exclude group: 'co.cask.tephra'
+      exclude group: 'com.google.code.findbugs', module: 'jsr305'
+      exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all'
+      exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet'
+      exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle'
+      exclude group: 'com.tdunning', module: 'json'
+      exclude group: 'javax.transaction', module: 'transaction-api'
+      exclude group: 'com.zaxxer', module: 'HikariCP'
+    }
+  }
+}
+
+project(':iceberg-flink:iceberg-flink-1.14-runtime') {
+  apply plugin: 'com.github.johnrengelman.shadow'
+
+  tasks.jar.dependsOn tasks.shadowJar
+
+  configurations {
+    implementation {
+      exclude group: 'org.apache.flink'
+      // included in Flink
+      exclude group: 'org.slf4j'
+      exclude group: 'org.apache.commons'
+      exclude group: 'commons-pool'
+      exclude group: 'commons-codec'
+      exclude group: 'org.xerial.snappy'
+      exclude group: 'javax.xml.bind'
+      exclude group: 'javax.annotation'
+    }
+  }
+
+  dependencies {
+    implementation project(':iceberg-flink:iceberg-flink-1.14')
+    implementation project(':iceberg-aws')
+    implementation(project(':iceberg-nessie')) {
+      exclude group: 'com.google.code.findbugs', module: 'jsr305'
+    }
+  }
+
+  shadowJar {
+    configurations = [project.configurations.runtimeClasspath]
+
+    zip64 true
+
+    // include the LICENSE and NOTICE files for the shaded Jar
+    from(projectDir) {
+      include 'LICENSE'
+      include 'NOTICE'
+    }
+
+    // Relocate dependencies to avoid conflicts
+    relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro'
+    relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet'
+    relocate 'com.google', 'org.apache.iceberg.shaded.com.google'
+    relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml'
+    relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes'
+    relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework'
+    relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded'
+    relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc'
+    relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift'
+    relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra'
+
+    classifier null
+  }
+
+  jar {
+    enabled = false
+  }
+}
diff --git a/flink/v1.14/flink-runtime/LICENSE b/flink/v1.14/flink-runtime/LICENSE
new file mode 100644
index 0000000..63df906
--- /dev/null
+++ b/flink/v1.14/flink-runtime/LICENSE
@@ -0,0 +1,484 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Apache Avro.
+
+Copyright: 2014-2020 The Apache Software Foundation.
+Home page: https://parquet.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains the Jackson JSON processor.
+
+Copyright: 2007-2020 Tatu Saloranta and other contributors
+Home page: http://jackson.codehaus.org/
+License: http://www.apache.org/licenses/LICENSE-2.0.txt
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Apache Parquet.
+
+Copyright: 2014-2020 The Apache Software Foundation.
+Home page: https://parquet.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Apache Thrift.
+
+Copyright: 2006-2010 The Apache Software Foundation.
+Home page: https://thrift.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains fastutil.
+
+Copyright: 2002-2014 Sebastiano Vigna
+Home page: http://fastutil.di.unimi.it/
+License: http://www.apache.org/licenses/LICENSE-2.0.html
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Apache ORC.
+
+Copyright: 2013-2020 The Apache Software Foundation.
+Home page: https://orc.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Apache Hive's storage API via ORC.
+
+Copyright: 2013-2020 The Apache Software Foundation.
+Home page: https://hive.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Airlift Aircompressor.
+
+Copyright: 2011-2020 Aircompressor authors.
+Home page: https://github.com/airlift/aircompressor
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Airlift Slice.
+
+Copyright: 2013-2020 Slice authors.
+Home page: https://github.com/airlift/slice
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains JetBrains annotations.
+
+Copyright: 2000-2020 JetBrains s.r.o.
+Home page: https://github.com/JetBrains/java-annotations
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Google Guava.
+
+Copyright: 2006-2020 The Guava Authors
+Home page: https://github.com/google/guava
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Google Error Prone Annotations.
+
+Copyright: Copyright 2011-2019 The Error Prone Authors
+Home page: https://github.com/google/error-prone
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains checkerframework checker-qual Annotations.
+
+Copyright: 2004-2020 the Checker Framework developers
+Home page: https://github.com/typetools/checker-framework
+License: https://github.com/typetools/checker-framework/blob/master/LICENSE.txt (MIT license)
+
+License text:
+| The annotations are licensed under the MIT License.  (The text of this
+| license appears below.)  More specifically, all the parts of the Checker
+| Framework that you might want to include with your own program use the
+| MIT License.  This is the checker-qual.jar file and all the files that
+| appear in it:  every file in a qual/ directory, plus utility files such
+| as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc.
+| In addition, the cleanroom implementations of third-party annotations,
+| which the Checker Framework recognizes as aliases for its own
+| annotations, are licensed under the MIT License.
+|
+| Permission is hereby granted, free of charge, to any person obtaining a copy
+| of this software and associated documentation files (the "Software"), to deal
+| in the Software without restriction, including without limitation the rights
+| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+| copies of the Software, and to permit persons to whom the Software is
+| furnished to do so, subject to the following conditions:
+|
+| The above copyright notice and this permission notice shall be included in
+| all copies or substantial portions of the Software.
+|
+| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+| THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Animal Sniffer Annotations.
+
+Copyright: 2009-2018 codehaus.org
+Home page: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/
+License: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/license.html (MIT license)
+
+License text:
+| The MIT License
+|
+| Copyright (c) 2009 codehaus.org.
+|
+| Permission is hereby granted, free of charge, to any person obtaining a copy
+| of this software and associated documentation files (the "Software"), to deal
+| in the Software without restriction, including without limitation the rights
+| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+| copies of the Software, and to permit persons to whom the Software is
+| furnished to do so, subject to the following conditions:
+|
+| The above copyright notice and this permission notice shall be included in
+| all copies or substantial portions of the Software.
+|
+| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+| THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Caffeine by Ben Manes.
+
+Copyright: 2014-2020 Ben Manes and contributors
+Home page: https://github.com/ben-manes/caffeine
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Apache Yetus audience annotations.
+
+Copyright: 2008-2020 The Apache Software Foundation.
+Home page: https://yetus.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains Google protobuf.
+
+Copyright: 2008 Google Inc.
+Home page: https://developers.google.com/protocol-buffers
+License: https://github.com/protocolbuffers/protobuf/blob/master/LICENSE (BSD)
+
+License text:
+
+| Copyright 2008 Google Inc.  All rights reserved.
+|
+| Redistribution and use in source and binary forms, with or without
+| modification, are permitted provided that the following conditions are
+| met:
+|
+|     * Redistributions of source code must retain the above copyright
+| notice, this list of conditions and the following disclaimer.
+|     * Redistributions in binary form must reproduce the above
+| copyright notice, this list of conditions and the following disclaimer
+| in the documentation and/or other materials provided with the
+| distribution.
+|     * Neither the name of Google Inc. nor the names of its
+| contributors may be used to endorse or promote products derived from
+| this software without specific prior written permission.
+|
+| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+|
+| Code generated by the Protocol Buffer compiler is owned by the owner
+| of the input file used when generating it.  This code is not
+| standalone and requires a support library to be linked with it.  This
+| support library is itself covered by the above license.
+
+--------------------------------------------------------------------------------
+
+This binary artifact contains ThreeTen.
+
+Copyright: 2007-present, Stephen Colebourne & Michael Nascimento Santos.
+Home page: https://www.threeten.org/threeten-extra/
+License: https://github.com/ThreeTen/threeten-extra/blob/master/LICENSE.txt (BSD 3-clause)
+
+License text:
+
+| All rights reserved.
+|
+| * Redistribution and use in source and binary forms, with or without
+|   modification, are permitted provided that the following conditions are met:
+|
+| * Redistributions of source code must retain the above copyright notice,
+|   this list of conditions and the following disclaimer.
+|
+| * Redistributions in binary form must reproduce the above copyright notice,
+|   this list of conditions and the following disclaimer in the documentation
+|   and/or other materials provided with the distribution.
+|
+| * Neither the name of JSR-310 nor the names of its contributors
+|   may be used to endorse or promote products derived from this software
+|   without specific prior written permission.
+|
+| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+| CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+| EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+| PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+| PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+| LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+| NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+| SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+This binary artifact includes Project Nessie with the following in its NOTICE
+file:
+
+| Dremio
+| Copyright 2015-2017 Dremio Corporation
+|
+| This product includes software developed at
+| The Apache Software Foundation (http://www.apache.org/).
+
+--------------------------------------------------------------------------------
+
+This binary includes code from Apache Commons.
+
+* Core ArrayUtil.
+
+Copyright: 2020 The Apache Software Foundation
+Home page: https://commons.apache.org/
+License: https://www.apache.org/licenses/LICENSE-2.0
diff --git a/flink/v1.14/flink-runtime/NOTICE b/flink/v1.14/flink-runtime/NOTICE
new file mode 100644
index 0000000..24a09ee
--- /dev/null
+++ b/flink/v1.14/flink-runtime/NOTICE
@@ -0,0 +1,91 @@
+
+Apache Iceberg
+Copyright 2017-2021 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+--------------------------------------------------------------------------------
+
+This binary artifact includes Apache ORC with the following in its NOTICE file:
+
+| Apache ORC
+| Copyright 2013-2019 The Apache Software Foundation
+|
+| This product includes software developed by The Apache Software
+| Foundation (http://www.apache.org/).
+|
+| This product includes software developed by Hewlett-Packard:
+| (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P
+
+--------------------------------------------------------------------------------
+
+This binary artifact includes Airlift Aircompressor with the following in its
+NOTICE file:
+
+| Snappy Copyright Notices
+| =========================
+|
+| * Copyright 2011 Dain Sundstrom <da...@iq80.com>
+| * Copyright 2011, Google Inc.<op...@google.com>
+|
+|
+| Snappy License
+| ===============
+| Copyright 2011, Google Inc.
+| All rights reserved.
+|
+| Redistribution and use in source and binary forms, with or without
+| modification, are permitted provided that the following conditions are
+| met:
+|
+|     * Redistributions of source code must retain the above copyright
+| notice, this list of conditions and the following disclaimer.
+|     * Redistributions in binary form must reproduce the above
+| copyright notice, this list of conditions and the following disclaimer
+| in the documentation and/or other materials provided with the
+| distribution.
+|     * Neither the name of Google Inc. nor the names of its
+| contributors may be used to endorse or promote products derived from
+| this software without specific prior written permission.
+|
+| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+--------------------------------------------------------------------------------
+
+This binary artifact includes Apache Yetus with the following in its NOTICE
+file:
+
+| Apache Yetus
+| Copyright 2008-2020 The Apache Software Foundation
+|
+| This product includes software developed at
+| The Apache Software Foundation (https://www.apache.org/).
+|
+| ---
+| Additional licenses for the Apache Yetus Source/Website:
+| ---
+|
+|
+| See LICENSE for terms.
+
+--------------------------------------------------------------------------------
+
+This binary artifact includes Project Nessie with the following in its NOTICE
+file:
+
+| Dremio
+| Copyright 2015-2017 Dremio Corporation
+|
+| This product includes software developed at
+| The Apache Software Foundation (http://www.apache.org/).
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java
new file mode 100644
index 0000000..1d53586
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+/**
+ * Serializable loader to load an Iceberg {@link Catalog}.
+ */
+public interface CatalogLoader extends Serializable {
+
+  /**
+   * Create a new catalog with the provided properties. NOTICE: for flink, we may initialize the {@link CatalogLoader}
+   * at flink sql client side or job manager side, and then serialize this catalog loader to task manager, finally
+   * deserialize it and create a new catalog at task manager side.
+   *
+   * @return a newly created {@link Catalog}
+   */
+  Catalog loadCatalog();
+
+  static CatalogLoader hadoop(String name, Configuration hadoopConf, Map<String, String> properties) {
+    return new HadoopCatalogLoader(name, hadoopConf, properties);
+  }
+
+  static CatalogLoader hive(String name, Configuration hadoopConf, Map<String, String> properties) {
+    return new HiveCatalogLoader(name, hadoopConf, properties);
+  }
+
+  static CatalogLoader custom(String name, Map<String, String> properties, Configuration hadoopConf, String impl) {
+    return new CustomCatalogLoader(name, properties, hadoopConf, impl);
+  }
+
+  class HadoopCatalogLoader implements CatalogLoader {
+    private final String catalogName;
+    private final SerializableConfiguration hadoopConf;
+    private final String warehouseLocation;
+    private final Map<String, String> properties;
+
+    private HadoopCatalogLoader(
+        String catalogName,
+        Configuration conf,
+        Map<String, String> properties) {
+      this.catalogName = catalogName;
+      this.hadoopConf = new SerializableConfiguration(conf);
+      this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+      this.properties = Maps.newHashMap(properties);
+    }
+
+    @Override
+    public Catalog loadCatalog() {
+      return CatalogUtil.loadCatalog(HadoopCatalog.class.getName(), catalogName, properties, hadoopConf.get());
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+          .add("catalogName", catalogName)
+          .add("warehouseLocation", warehouseLocation)
+          .toString();
+    }
+  }
+
+  class HiveCatalogLoader implements CatalogLoader {
+    private final String catalogName;
+    private final SerializableConfiguration hadoopConf;
+    private final String uri;
+    private final String warehouse;
+    private final int clientPoolSize;
+    private final Map<String, String> properties;
+
+    private HiveCatalogLoader(String catalogName, Configuration conf, Map<String, String> properties) {
+      this.catalogName = catalogName;
+      this.hadoopConf = new SerializableConfiguration(conf);
+      this.uri = properties.get(CatalogProperties.URI);
+      this.warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+      this.clientPoolSize = properties.containsKey(CatalogProperties.CLIENT_POOL_SIZE) ?
+          Integer.parseInt(properties.get(CatalogProperties.CLIENT_POOL_SIZE)) :
+          CatalogProperties.CLIENT_POOL_SIZE_DEFAULT;
+      this.properties = Maps.newHashMap(properties);
+    }
+
+    @Override
+    public Catalog loadCatalog() {
+      return CatalogUtil.loadCatalog(HiveCatalog.class.getName(), catalogName, properties, hadoopConf.get());
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+          .add("catalogName", catalogName)
+          .add("uri", uri)
+          .add("warehouse", warehouse)
+          .add("clientPoolSize", clientPoolSize)
+          .toString();
+    }
+  }
+
+  class CustomCatalogLoader implements CatalogLoader {
+
+    private final SerializableConfiguration hadoopConf;
+    private final Map<String, String> properties;
+    private final String name;
+    private final String impl;
+
+    private CustomCatalogLoader(
+        String name,
+        Map<String, String> properties,
+        Configuration conf,
+        String impl) {
+      this.hadoopConf = new SerializableConfiguration(conf);
+      this.properties = Maps.newHashMap(properties); // wrap into a hashmap for serialization
+      this.name = name;
+      this.impl = Preconditions.checkNotNull(impl, "Cannot initialize custom Catalog, impl class name is null");
+    }
+
+    @Override
+    public Catalog loadCatalog() {
+      return CatalogUtil.loadCatalog(impl, name, properties, hadoopConf.get());
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+          .add("name", name)
+          .add("impl", impl)
+          .toString();
+    }
+  }
+
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
new file mode 100644
index 0000000..f30cc44
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
@@ -0,0 +1,724 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.factories.Factory;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.Transaction;
+import org.apache.iceberg.UpdateProperties;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * <p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {
+
+  private final CatalogLoader catalogLoader;
+  private final Catalog icebergCatalog;
+  private final Namespace baseNamespace;
+  private final SupportsNamespaces asNamespaceCatalog;
+  private final Closeable closeable;
+  private final boolean cacheEnabled;
+
+  public FlinkCatalog(
+      String catalogName,
+      String defaultDatabase,
+      Namespace baseNamespace,
+      CatalogLoader catalogLoader,
+      boolean cacheEnabled) {
+    super(catalogName, defaultDatabase);
+    this.catalogLoader = catalogLoader;
+    this.baseNamespace = baseNamespace;
+    this.cacheEnabled = cacheEnabled;
+
+    Catalog originalCatalog = catalogLoader.loadCatalog();
+    icebergCatalog = cacheEnabled ? CachingCatalog.wrap(originalCatalog) : originalCatalog;
+    asNamespaceCatalog = originalCatalog instanceof SupportsNamespaces ? (SupportsNamespaces) originalCatalog : null;
+    closeable = originalCatalog instanceof Closeable ? (Closeable) originalCatalog : null;
+  }
+
+  @Override
+  public void open() throws CatalogException {
+    // Create the default database if it does not exist.
+    try {
+      createDatabase(getDefaultDatabase(), ImmutableMap.of(), true);
+    } catch (DatabaseAlreadyExistException e) {
+      // Ignore the exception if it's already exist.
+    }
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (closeable != null) {
+      try {
+        closeable.close();
+      } catch (IOException e) {
+        throw new CatalogException(e);
+      }
+    }
+  }
+
+  public Catalog catalog() {
+    return icebergCatalog;
+  }
+
+  private Namespace toNamespace(String database) {
+    String[] namespace = new String[baseNamespace.levels().length + 1];
+    System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length);
+    namespace[baseNamespace.levels().length] = database;
+    return Namespace.of(namespace);
+  }
+
+  TableIdentifier toIdentifier(ObjectPath path) {
+    return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
+  }
+
+  @Override
+  public List<String> listDatabases() throws CatalogException {
+    if (asNamespaceCatalog == null) {
+      return Collections.singletonList(getDefaultDatabase());
+    }
+
+    return asNamespaceCatalog.listNamespaces(baseNamespace).stream()
+        .map(n -> n.level(n.levels().length - 1))
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog == null) {
+      if (!getDefaultDatabase().equals(databaseName)) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      } else {
+        return new CatalogDatabaseImpl(Maps.newHashMap(), "");
+      }
+    } else {
+      try {
+        Map<String, String> metadata =
+            Maps.newHashMap(asNamespaceCatalog.loadNamespaceMetadata(toNamespace(databaseName)));
+        String comment = metadata.remove("comment");
+        return new CatalogDatabaseImpl(metadata, comment);
+      } catch (NoSuchNamespaceException e) {
+        throw new DatabaseNotExistException(getName(), databaseName, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean databaseExists(String databaseName) throws CatalogException {
+    try {
+      getDatabase(databaseName);
+      return true;
+    } catch (DatabaseNotExistException ignore) {
+      return false;
+    }
+  }
+
+  @Override
+  public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    createDatabase(name, mergeComment(database.getProperties(), database.getComment()), ignoreIfExists);
+  }
+
+  private void createDatabase(String databaseName, Map<String, String> metadata, boolean ignoreIfExists)
+      throws DatabaseAlreadyExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        asNamespaceCatalog.createNamespace(toNamespace(databaseName), metadata);
+      } catch (AlreadyExistsException e) {
+        if (!ignoreIfExists) {
+          throw new DatabaseAlreadyExistException(getName(), databaseName, e);
+        }
+      }
+    } else {
+      throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + getName());
+    }
+  }
+
+  private Map<String, String> mergeComment(Map<String, String> metadata, String comment) {
+    Map<String, String> ret = Maps.newHashMap(metadata);
+    if (metadata.containsKey("comment")) {
+      throw new CatalogException("Database properties should not contain key: 'comment'.");
+    }
+
+    if (!StringUtils.isNullOrWhitespaceOnly(comment)) {
+      ret.put("comment", comment);
+    }
+    return ret;
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      try {
+        boolean success = asNamespaceCatalog.dropNamespace(toNamespace(name));
+        if (!success && !ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name);
+        }
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      } catch (NamespaceNotEmptyException e) {
+        throw new DatabaseNotEmptyException(getName(), name, e);
+      }
+    } else {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    if (asNamespaceCatalog != null) {
+      Namespace namespace = toNamespace(name);
+      Map<String, String> updates = Maps.newHashMap();
+      Set<String> removals = Sets.newHashSet();
+
+      try {
+        Map<String, String> oldProperties = asNamespaceCatalog.loadNamespaceMetadata(namespace);
+        Map<String, String> newProperties = mergeComment(newDatabase.getProperties(), newDatabase.getComment());
+
+        for (String key : oldProperties.keySet()) {
+          if (!newProperties.containsKey(key)) {
+            removals.add(key);
+          }
+        }
+
+        for (Map.Entry<String, String> entry : newProperties.entrySet()) {
+          if (!entry.getValue().equals(oldProperties.get(entry.getKey()))) {
+            updates.put(entry.getKey(), entry.getValue());
+          }
+        }
+
+        if (!updates.isEmpty()) {
+          asNamespaceCatalog.setProperties(namespace, updates);
+        }
+
+        if (!removals.isEmpty()) {
+          asNamespaceCatalog.removeProperties(namespace, removals);
+        }
+
+      } catch (NoSuchNamespaceException e) {
+        if (!ignoreIfNotExists) {
+          throw new DatabaseNotExistException(getName(), name, e);
+        }
+      }
+    } else {
+      if (getDefaultDatabase().equals(name)) {
+        throw new CatalogException(
+            "Can not alter the default database when the iceberg catalog doesn't support namespaces.");
+      }
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    }
+  }
+
+  @Override
+  public List<String> listTables(String databaseName) throws DatabaseNotExistException, CatalogException {
+    try {
+      return icebergCatalog.listTables(toNamespace(databaseName)).stream()
+          .map(TableIdentifier::name)
+          .collect(Collectors.toList());
+    } catch (NoSuchNamespaceException e) {
+      throw new DatabaseNotExistException(getName(), databaseName, e);
+    }
+  }
+
+  @Override
+  public CatalogTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException {
+    Table table = loadIcebergTable(tablePath);
+    return toCatalogTable(table);
+  }
+
+  private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException {
+    try {
+      Table table = icebergCatalog.loadTable(toIdentifier(tablePath));
+      if (cacheEnabled) {
+        table.refresh();
+      }
+
+      return table;
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new TableNotExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    return icebergCatalog.tableExists(toIdentifier(tablePath));
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    try {
+      icebergCatalog.dropTable(toIdentifier(tablePath));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      if (!ignoreIfNotExists) {
+        throw new TableNotExistException(getName(), tablePath, e);
+      }
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    try {
+      icebergCatalog.renameTable(
+          toIdentifier(tablePath),
+          toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName)));
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      if (!ignoreIfNotExists) {
+        throw new TableNotExistException(getName(), tablePath, e);
+      }
+    } catch (AlreadyExistsException e) {
+      throw new TableAlreadyExistException(getName(), tablePath, e);
+    }
+  }
+
+  @Override
+  public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists)
+      throws CatalogException, TableAlreadyExistException {
+    if (Objects.equals(table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) {
+      throw new IllegalArgumentException("Cannot create the table with 'connector'='iceberg' table property in " +
+          "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " +
+          "create table without 'connector'='iceberg' related properties in an iceberg table.");
+    }
+
+    createIcebergTable(tablePath, table, ignoreIfExists);
+  }
+
+  void createIcebergTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists)
+      throws CatalogException, TableAlreadyExistException {
+    validateFlinkTable(table);
+
+    Schema icebergSchema = FlinkSchemaUtil.convert(table.getSchema());
+    PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema);
+
+    ImmutableMap.Builder<String, String> properties = ImmutableMap.builder();
+    String location = null;
+    for (Map.Entry<String, String> entry : table.getOptions().entrySet()) {
+      if ("location".equalsIgnoreCase(entry.getKey())) {
+        location = entry.getValue();
+      } else {
+        properties.put(entry.getKey(), entry.getValue());
+      }
+    }
+
+    try {
+      icebergCatalog.createTable(
+          toIdentifier(tablePath),
+          icebergSchema,
+          spec,
+          location,
+          properties.build());
+    } catch (AlreadyExistsException e) {
+      if (!ignoreIfExists) {
+        throw new TableAlreadyExistException(getName(), tablePath, e);
+      }
+    }
+  }
+
+  @Override
+  public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists)
+      throws CatalogException, TableNotExistException {
+    validateFlinkTable(newTable);
+
+    Table icebergTable;
+    try {
+      icebergTable = loadIcebergTable(tablePath);
+    } catch (TableNotExistException e) {
+      if (!ignoreIfNotExists) {
+        throw e;
+      } else {
+        return;
+      }
+    }
+
+    CatalogTable table = toCatalogTable(icebergTable);
+
+    // Currently, Flink SQL only support altering table properties.
+
+    // For current Flink Catalog API, support for adding/removing/renaming columns cannot be done by comparing
+    // CatalogTable instances, unless the Flink schema contains Iceberg column IDs.
+    if (!table.getSchema().equals(newTable.getSchema())) {
+      throw new UnsupportedOperationException("Altering schema is not supported yet.");
+    }
+
+    if (!table.getPartitionKeys().equals(((CatalogTable) newTable).getPartitionKeys())) {
+      throw new UnsupportedOperationException("Altering partition keys is not supported yet.");
+    }
+
+    Map<String, String> oldProperties = table.getOptions();
+    Map<String, String> setProperties = Maps.newHashMap();
+
+    String setLocation = null;
+    String setSnapshotId = null;
+    String pickSnapshotId = null;
+
+    for (Map.Entry<String, String> entry : newTable.getOptions().entrySet()) {
+      String key = entry.getKey();
+      String value = entry.getValue();
+
+      if (Objects.equals(value, oldProperties.get(key))) {
+        continue;
+      }
+
+      if ("location".equalsIgnoreCase(key)) {
+        setLocation = value;
+      } else if ("current-snapshot-id".equalsIgnoreCase(key)) {
+        setSnapshotId = value;
+      } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(key)) {
+        pickSnapshotId = value;
+      } else {
+        setProperties.put(key, value);
+      }
+    }
+
+    oldProperties.keySet().forEach(k -> {
+      if (!newTable.getOptions().containsKey(k)) {
+        setProperties.put(k, null);
+      }
+    });
+
+    commitChanges(icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties);
+  }
+
+  private static void validateFlinkTable(CatalogBaseTable table) {
+    Preconditions.checkArgument(table instanceof CatalogTable, "The Table should be a CatalogTable.");
+
+    TableSchema schema = table.getSchema();
+    schema.getTableColumns().forEach(column -> {
+      if (!FlinkCompatibilityUtil.isPhysicalColumn(column)) {
+        throw new UnsupportedOperationException("Creating table with computed columns is not supported yet.");
+      }
+    });
+
+    if (!schema.getWatermarkSpecs().isEmpty()) {
+      throw new UnsupportedOperationException("Creating table with watermark specs is not supported yet.");
+    }
+  }
+
+  private static PartitionSpec toPartitionSpec(List<String> partitionKeys, Schema icebergSchema) {
+    PartitionSpec.Builder builder = PartitionSpec.builderFor(icebergSchema);
+    partitionKeys.forEach(builder::identity);
+    return builder.build();
+  }
+
+  private static List<String> toPartitionKeys(PartitionSpec spec, Schema icebergSchema) {
+    List<String> partitionKeys = Lists.newArrayList();
+    for (PartitionField field : spec.fields()) {
+      if (field.transform().isIdentity()) {
+        partitionKeys.add(icebergSchema.findColumnName(field.sourceId()));
+      } else {
+        // Not created by Flink SQL.
+        // For compatibility with iceberg tables, return empty.
+        // TODO modify this after Flink support partition transform.
+        return Collections.emptyList();
+      }
+    }
+    return partitionKeys;
+  }
+
+  private static void commitChanges(Table table, String setLocation, String setSnapshotId,
+                                    String pickSnapshotId, Map<String, String> setProperties) {
+    // don't allow setting the snapshot and picking a commit at the same time because order is ambiguous and choosing
+    // one order leads to different results
+    Preconditions.checkArgument(setSnapshotId == null || pickSnapshotId == null,
+        "Cannot set the current snapshot ID and cherry-pick snapshot changes");
+
+    if (setSnapshotId != null) {
+      long newSnapshotId = Long.parseLong(setSnapshotId);
+      table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit();
+    }
+
+    // if updating the table snapshot, perform that update first in case it fails
+    if (pickSnapshotId != null) {
+      long newSnapshotId = Long.parseLong(pickSnapshotId);
+      table.manageSnapshots().cherrypick(newSnapshotId).commit();
+    }
+
+    Transaction transaction = table.newTransaction();
+
+    if (setLocation != null) {
+      transaction.updateLocation()
+          .setLocation(setLocation)
+          .commit();
+    }
+
+    if (!setProperties.isEmpty()) {
+      UpdateProperties updateProperties = transaction.updateProperties();
+      setProperties.forEach((k, v) -> {
+        if (v == null) {
+          updateProperties.remove(k);
+        } else {
+          updateProperties.set(k, v);
+        }
+      });
+      updateProperties.commit();
+    }
+
+    transaction.commitTransaction();
+  }
+
+  static CatalogTable toCatalogTable(Table table) {
+    TableSchema schema = FlinkSchemaUtil.toSchema(table.schema());
+    List<String> partitionKeys = toPartitionKeys(table.spec(), table.schema());
+
+    // NOTE: We can not create a IcebergCatalogTable extends CatalogTable, because Flink optimizer may use
+    // CatalogTableImpl to copy a new catalog table.
+    // Let's re-loading table from Iceberg catalog when creating source/sink operators.
+    // Iceberg does not have Table comment, so pass a null (Default comment value in Flink).
+    return new CatalogTableImpl(schema, partitionKeys, table.properties(), null);
+  }
+
+  @Override
+  public Optional<Factory> getFactory() {
+    return Optional.of(new FlinkDynamicTableFactory(this));
+  }
+
+  CatalogLoader getCatalogLoader() {
+    return catalogLoader;
+  }
+
+  // ------------------------------ Unsupported methods ---------------------------------------------
+
+  @Override
+  public List<String> listViews(String databaseName) throws CatalogException {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void createPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition partition,
+      boolean ignoreIfExists) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void dropPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void alterPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition newPartition,
+      boolean ignoreIfNotExists) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<String> listFunctions(String dbName) throws CatalogException {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public CatalogFunction getFunction(ObjectPath functionPath) throws FunctionNotExistException, CatalogException {
+    throw new FunctionNotExistException(getName(), functionPath);
+  }
+
+  @Override
+  public boolean functionExists(ObjectPath functionPath) throws CatalogException {
+    return false;
+  }
+
+  @Override
+  public void createFunction(ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void alterFunction(ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void alterTableStatistics(ObjectPath tablePath, CatalogTableStatistics tableStatistics,
+      boolean ignoreIfNotExists) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void alterTableColumnStatistics(ObjectPath tablePath, CatalogColumnStatistics columnStatistics,
+      boolean ignoreIfNotExists) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void alterPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec,
+      CatalogTableStatistics partitionStatistics, boolean ignoreIfNotExists) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void alterPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec,
+      CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath)
+      throws TableNotExistException, TableNotPartitionedException, CatalogException {
+    Table table = loadIcebergTable(tablePath);
+
+    if (table.spec().isUnpartitioned()) {
+      throw new TableNotPartitionedException(icebergCatalog.name(), tablePath);
+    }
+
+    Set<CatalogPartitionSpec> set = Sets.newHashSet();
+    try (CloseableIterable<FileScanTask> tasks = table.newScan().planFiles()) {
+      for (DataFile dataFile : CloseableIterable.transform(tasks, FileScanTask::file)) {
+        Map<String, String> map = Maps.newHashMap();
+        StructLike structLike = dataFile.partition();
+        PartitionSpec spec = table.specs().get(dataFile.specId());
+        for (int i = 0; i < structLike.size(); i++) {
+          map.put(spec.fields().get(i).name(), String.valueOf(structLike.get(i, Object.class)));
+        }
+        set.add(new CatalogPartitionSpec(map));
+      }
+    } catch (IOException e) {
+      throw new CatalogException(String.format("Failed to list partitions of table %s", tablePath), e);
+    }
+
+    return Lists.newArrayList(set);
+  }
+
+  @Override
+  public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<CatalogPartitionSpec> listPartitionsByFilter(ObjectPath tablePath, List<Expression> filters)
+      throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  // After partition pruning and filter push down, the statistics have become very inaccurate, so the statistics from
+  // here are of little significance.
+  // Flink will support something like SupportsReportStatistics in future.
+
+  @Override
+  public CatalogTableStatistics getTableStatistics(ObjectPath tablePath)
+      throws CatalogException {
+    return CatalogTableStatistics.UNKNOWN;
+  }
+
+  @Override
+  public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath)
+      throws CatalogException {
+    return CatalogColumnStatistics.UNKNOWN;
+  }
+
+  @Override
+  public CatalogTableStatistics getPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+      throws CatalogException {
+    return CatalogTableStatistics.UNKNOWN;
+  }
+
+  @Override
+  public CatalogColumnStatistics getPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+      throws CatalogException {
+    return CatalogColumnStatistics.UNKNOWN;
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
new file mode 100644
index 0000000..ffa54c0
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
@@ -0,0 +1,162 @@
+/*
+ * 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;
+
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.runtime.util.HadoopUtils;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.factories.CatalogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+/**
+ * A Flink Catalog factory implementation that creates {@link FlinkCatalog}.
+ * <p>
+ * This supports the following catalog configuration options:
+ * <ul>
+ *   <li><code>type</code> - Flink catalog factory key, should be "iceberg"</li>
+ *   <li><code>catalog-type</code> - iceberg catalog type, "hive" or "hadoop"</li>
+ *   <li><code>uri</code> - the Hive Metastore URI (Hive catalog only)</li>
+ *   <li><code>clients</code> - the Hive Client Pool Size (Hive catalog only)</li>
+ *   <li><code>warehouse</code> - the warehouse path (Hadoop catalog only)</li>
+ *   <li><code>default-database</code> - a database name to use as the default</li>
+ *   <li><code>base-namespace</code> - a base namespace as the prefix for all databases (Hadoop catalog only)</li>
+ *   <li><code>cache-enabled</code> - whether to enable catalog cache</li>
+ * </ul>
+ * <p>
+ * To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override
+ * {@link #createCatalogLoader(String, Map, Configuration)}.
+ */
+public class FlinkCatalogFactory implements CatalogFactory {
+
+  // Can not just use "type", it conflicts with CATALOG_TYPE.
+  public static final String ICEBERG_CATALOG_TYPE = "catalog-type";
+  public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop";
+  public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive";
+
+  public static final String HIVE_CONF_DIR = "hive-conf-dir";
+  public static final String DEFAULT_DATABASE = "default-database";
+  public static final String DEFAULT_DATABASE_NAME = "default";
+  public static final String BASE_NAMESPACE = "base-namespace";
+  public static final String CACHE_ENABLED = "cache-enabled";
+
+  public static final String TYPE = "type";
+  public static final String PROPERTY_VERSION = "property-version";
+
+  /**
+   * Create an Iceberg {@link org.apache.iceberg.catalog.Catalog} loader to be used by this Flink catalog adapter.
+   *
+   * @param name       Flink's catalog name
+   * @param properties Flink's catalog properties
+   * @param hadoopConf Hadoop configuration for catalog
+   * @return an Iceberg catalog loader
+   */
+  static CatalogLoader createCatalogLoader(String name, Map<String, String> properties, Configuration hadoopConf) {
+    String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL);
+    if (catalogImpl != null) {
+      String catalogType = properties.get(ICEBERG_CATALOG_TYPE);
+      Preconditions.checkArgument(catalogType == null,
+          "Cannot create catalog %s, both catalog-type and catalog-impl are set: catalog-type=%s, catalog-impl=%s",
+          name, catalogType, catalogImpl);
+      return CatalogLoader.custom(name, properties, hadoopConf, catalogImpl);
+    }
+
+    String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HIVE);
+    switch (catalogType.toLowerCase(Locale.ENGLISH)) {
+      case ICEBERG_CATALOG_TYPE_HIVE:
+        // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in that case it will
+        // fallback to parse those values from hadoop configuration which is loaded from classpath.
+        String hiveConfDir = properties.get(HIVE_CONF_DIR);
+        Configuration newHadoopConf = mergeHiveConf(hadoopConf, hiveConfDir);
+        return CatalogLoader.hive(name, newHadoopConf, properties);
+
+      case ICEBERG_CATALOG_TYPE_HADOOP:
+        return CatalogLoader.hadoop(name, hadoopConf, properties);
+
+      default:
+        throw new UnsupportedOperationException("Unknown catalog-type: " + catalogType +
+            " (Must be 'hive' or 'hadoop')");
+    }
+  }
+
+  @Override
+  public Map<String, String> requiredContext() {
+    Map<String, String> context = Maps.newHashMap();
+    context.put(TYPE, "iceberg");
+    context.put(PROPERTY_VERSION, "1");
+    return context;
+  }
+
+  @Override
+  public List<String> supportedProperties() {
+    return ImmutableList.of("*");
+  }
+
+  @Override
+  public Catalog createCatalog(String name, Map<String, String> properties) {
+    return createCatalog(name, properties, clusterHadoopConf());
+  }
+
+  protected Catalog createCatalog(String name, Map<String, String> properties, Configuration hadoopConf) {
+    CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf);
+    String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME);
+
+    Namespace baseNamespace = Namespace.empty();
+    if (properties.containsKey(BASE_NAMESPACE)) {
+      baseNamespace = Namespace.of(properties.get(BASE_NAMESPACE).split("\\."));
+    }
+
+    boolean cacheEnabled = Boolean.parseBoolean(properties.getOrDefault(CACHE_ENABLED, "true"));
+    return new FlinkCatalog(name, defaultDatabase, baseNamespace, catalogLoader, cacheEnabled);
+  }
+
+  private static Configuration mergeHiveConf(Configuration hadoopConf, String hiveConfDir) {
+    Configuration newConf = new Configuration(hadoopConf);
+    if (!Strings.isNullOrEmpty(hiveConfDir)) {
+      Preconditions.checkState(Files.exists(Paths.get(hiveConfDir, "hive-site.xml")),
+          "There should be a hive-site.xml file under the directory %s", hiveConfDir);
+      newConf.addResource(new Path(hiveConfDir, "hive-site.xml"));
+    } else {
+      // If don't provide the hive-site.xml path explicitly, it will try to load resource from classpath. If still
+      // couldn't load the configuration file, then it will throw exception in HiveCatalog.
+      URL configFile = CatalogLoader.class.getClassLoader().getResource("hive-site.xml");
+      if (configFile != null) {
+        newConf.addResource(configFile);
+      }
+    }
+    return newConf;
+  }
+
+  public static Configuration clusterHadoopConf() {
+    return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java
new file mode 100644
index 0000000..067abe8
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java
@@ -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.iceberg.flink;
+
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+public class FlinkConfigOptions {
+
+  private FlinkConfigOptions() {
+  }
+
+  public static final ConfigOption<Boolean> TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM =
+      ConfigOptions.key("table.exec.iceberg.infer-source-parallelism")
+          .booleanType()
+          .defaultValue(true)
+          .withDescription("If is false, parallelism of source are set by config.\n" +
+              "If is true, source parallelism is inferred according to splits number.\n");
+
+  public static final ConfigOption<Integer> TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX =
+      ConfigOptions.key("table.exec.iceberg.infer-source-parallelism.max")
+          .intType()
+          .defaultValue(100)
+          .withDescription("Sets max infer parallelism for source operator.");
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java
new file mode 100644
index 0000000..fc0269f
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java
@@ -0,0 +1,201 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.Set;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+public class FlinkDynamicTableFactory implements DynamicTableSinkFactory, DynamicTableSourceFactory {
+  static final String FACTORY_IDENTIFIER = "iceberg";
+
+  private static final ConfigOption<String> CATALOG_NAME =
+      ConfigOptions.key("catalog-name")
+          .stringType()
+          .noDefaultValue()
+          .withDescription("Catalog name");
+
+  private static final ConfigOption<String> CATALOG_TYPE =
+      ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE)
+          .stringType()
+          .noDefaultValue()
+          .withDescription("Catalog type, the optional types are: custom, hadoop, hive.");
+
+  private static final ConfigOption<String> CATALOG_DATABASE =
+      ConfigOptions.key("catalog-database")
+          .stringType()
+          .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME)
+          .withDescription("Database name managed in the iceberg catalog.");
+
+  private static final ConfigOption<String> CATALOG_TABLE =
+      ConfigOptions.key("catalog-table")
+          .stringType()
+          .noDefaultValue()
+          .withDescription("Table name managed in the underlying iceberg catalog and database.");
+
+  // Flink 1.13.x change the return type from CatalogTable interface to ResolvedCatalogTable which extends the
+  // CatalogTable. Here we use the dynamic method loading approach to avoid adding explicit CatalogTable or
+  // ResolvedCatalogTable class into the iceberg-flink-runtime jar for compatibility purpose.
+  private static final DynMethods.UnboundMethod GET_CATALOG_TABLE = DynMethods.builder("getCatalogTable")
+      .impl(Context.class, "getCatalogTable")
+      .orNoop()
+      .build();
+
+  private final FlinkCatalog catalog;
+
+  public FlinkDynamicTableFactory() {
+    this.catalog = null;
+  }
+
+  public FlinkDynamicTableFactory(FlinkCatalog catalog) {
+    this.catalog = catalog;
+  }
+
+  private static CatalogTable loadCatalogTable(Context context) {
+    return GET_CATALOG_TABLE.invoke(context);
+  }
+
+  @Override
+  public DynamicTableSource createDynamicTableSource(Context context) {
+    ObjectIdentifier objectIdentifier = context.getObjectIdentifier();
+    CatalogTable catalogTable = loadCatalogTable(context);
+    Map<String, String> tableProps = catalogTable.getOptions();
+    TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema());
+
+    TableLoader tableLoader;
+    if (catalog != null) {
+      tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath());
+    } else {
+      tableLoader = createTableLoader(catalogTable, tableProps, objectIdentifier.getDatabaseName(),
+          objectIdentifier.getObjectName());
+    }
+
+    return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration());
+  }
+
+  @Override
+  public DynamicTableSink createDynamicTableSink(Context context) {
+    ObjectPath objectPath = context.getObjectIdentifier().toObjectPath();
+    CatalogTable catalogTable = loadCatalogTable(context);
+    Map<String, String> tableProps = catalogTable.getOptions();
+    TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema());
+
+    TableLoader tableLoader;
+    if (catalog != null) {
+      tableLoader = createTableLoader(catalog, objectPath);
+    } else {
+      tableLoader = createTableLoader(catalogTable, tableProps, objectPath.getDatabaseName(),
+          objectPath.getObjectName());
+    }
+
+    return new IcebergTableSink(tableLoader, tableSchema);
+  }
+
+  @Override
+  public Set<ConfigOption<?>> requiredOptions() {
+    Set<ConfigOption<?>> options = Sets.newHashSet();
+    options.add(CATALOG_TYPE);
+    options.add(CATALOG_NAME);
+    return options;
+  }
+
+  @Override
+  public Set<ConfigOption<?>> optionalOptions() {
+    Set<ConfigOption<?>> options = Sets.newHashSet();
+    options.add(CATALOG_DATABASE);
+    options.add(CATALOG_TABLE);
+    return options;
+  }
+
+  @Override
+  public String factoryIdentifier() {
+    return FACTORY_IDENTIFIER;
+  }
+
+  private static TableLoader createTableLoader(CatalogBaseTable catalogBaseTable,
+                                               Map<String, String> tableProps,
+                                               String databaseName,
+                                               String tableName) {
+    Configuration flinkConf = new Configuration();
+    tableProps.forEach(flinkConf::setString);
+
+    String catalogName = flinkConf.getString(CATALOG_NAME);
+    Preconditions.checkNotNull(catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key());
+
+    String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName);
+    Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null");
+
+    String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName);
+    Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null");
+
+    org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf();
+    FlinkCatalogFactory factory = new FlinkCatalogFactory();
+    FlinkCatalog flinkCatalog = (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf);
+    ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable);
+
+    // Create database if not exists in the external catalog.
+    if (!flinkCatalog.databaseExists(catalogDatabase)) {
+      try {
+        flinkCatalog.createDatabase(catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true);
+      } catch (DatabaseAlreadyExistException e) {
+        throw new AlreadyExistsException(e, "Database %s already exists in the iceberg catalog %s.", catalogName,
+            catalogDatabase);
+      }
+    }
+
+    // Create table if not exists in the external catalog.
+    if (!flinkCatalog.tableExists(objectPath)) {
+      try {
+        flinkCatalog.createIcebergTable(objectPath, catalogBaseTable, true);
+      } catch (TableAlreadyExistException e) {
+        throw new AlreadyExistsException(e, "Table %s already exists in the database %s and catalog %s",
+            catalogTable, catalogDatabase, catalogName);
+      }
+    }
+
+    return TableLoader.fromCatalog(flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable));
+  }
+
+  private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) {
+    Preconditions.checkNotNull(catalog, "Flink catalog cannot be null");
+    return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath));
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
new file mode 100644
index 0000000..5e5c9c1
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
@@ -0,0 +1,252 @@
+/*
+ * 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;
+
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expression.Operation;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, Operation>builder()
+      .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ)
+      .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ)
+      .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT)
+      .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ)
+      .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT)
+      .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ)
+      .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL)
+      .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL)
+      .put(BuiltInFunctionDefinitions.AND, Operation.AND)
+      .put(BuiltInFunctionDefinitions.OR, Operation.OR)
+      .put(BuiltInFunctionDefinitions.NOT, Operation.NOT)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * Convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  public static Optional<Expression> convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (!(flinkExpression instanceof CallExpression)) {
+      return Optional.empty();
+    }
+
+    CallExpression call = (CallExpression) flinkExpression;
+    Operation op = FILTERS.get(call.getFunctionDefinition());
+    if (op != null) {
+      switch (op) {
+        case IS_NULL:
+          return onlyChildAs(call, FieldReferenceExpression.class)
+              .map(FieldReferenceExpression::getName)
+              .map(Expressions::isNull);
+
+        case NOT_NULL:
+          return onlyChildAs(call, FieldReferenceExpression.class)
+              .map(FieldReferenceExpression::getName)
+              .map(Expressions::notNull);
+
+        case LT:
+          return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertFieldAndLiteral(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertFieldAndLiteral(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return convertFieldAndLiteral((ref, lit) -> {
+            if (NaNUtil.isNaN(lit)) {
+              return Expressions.isNaN(ref);
+            } else {
+              return Expressions.equal(ref, lit);
+            }
+          }, call);
+
+        case NOT_EQ:
+          return convertFieldAndLiteral((ref, lit) -> {
+            if (NaNUtil.isNaN(lit)) {
+              return Expressions.notNaN(ref);
+            } else {
+              return Expressions.notEqual(ref, lit);
+            }
+          }, call);
+
+        case NOT:
+          return onlyChildAs(call, CallExpression.class).flatMap(FlinkFilters::convert).map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> onlyChildAs(CallExpression call,
+                                                                        Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args.size() != 2) {
+      return Optional.empty();
+    }
+
+    org.apache.flink.table.expressions.Expression left = args.get(0);
+    org.apache.flink.table.expressions.Expression right = args.get(1);
+
+    if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) {
+      String name = ((FieldReferenceExpression) left).getName();
+      return convertLiteral((ValueLiteralExpression) right).flatMap(lit -> {
+        if (lit instanceof String) {
+          String pattern = (String) lit;
+          Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+          // exclude special char of LIKE
+          // '_' is the wildcard of the SQL LIKE
+          if (!pattern.contains("_") && matcher.matches()) {
+            return Optional.of(Expressions.startsWith(name, matcher.group(1)));
+          }
+        }
+
+        return Optional.empty();
+      });
+    }
+
+    return Optional.empty();
+  }
+
+  private static Optional<Expression> convertLogicExpression(BiFunction<Expression, Expression, Expression> function,
+                                                             CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args == null || args.size() != 2) {
+      return Optional.empty();
+    }
+
+    Optional<Expression> left = convert(args.get(0));
+    Optional<Expression> right = convert(args.get(1));
+    if (left.isPresent() && right.isPresent()) {
+      return Optional.of(function.apply(left.get(), right.get()));
+    }
+
+    return Optional.empty();
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static Optional<Expression> convertFieldAndLiteral(BiFunction<String, Object, Expression> expr,
+                                                             CallExpression call) {
+    return convertFieldAndLiteral(expr, expr, call);
+  }
+
+  private static Optional<Expression> convertFieldAndLiteral(
+      BiFunction<String, Object, Expression> convertLR, BiFunction<String, Object, Expression> convertRL,
+      CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args.size() != 2) {
+      return Optional.empty();
+    }
+
+    org.apache.flink.table.expressions.Expression left = args.get(0);
+    org.apache.flink.table.expressions.Expression right = args.get(1);
+
+    if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) {
+      String name = ((FieldReferenceExpression) left).getName();
+      Optional<Object> lit = convertLiteral((ValueLiteralExpression) right);
+      if (lit.isPresent()) {
+        return Optional.of(convertLR.apply(name, lit.get()));
+      }
+    } else if (left instanceof ValueLiteralExpression && right instanceof FieldReferenceExpression) {
+      Optional<Object> lit = convertLiteral((ValueLiteralExpression) left);
+      String name = ((FieldReferenceExpression) right).getName();
+      if (lit.isPresent()) {
+        return Optional.of(convertRL.apply(name, lit.get()));
+      }
+    }
+
+    return Optional.empty();
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java
new file mode 100644
index 0000000..6501c02
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java
@@ -0,0 +1,51 @@
+/*
+ * 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;
+
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.FixupTypes;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+/**
+ * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one,
+ * which may not be correct.
+ */
+class FlinkFixupTypes extends FixupTypes {
+
+  private FlinkFixupTypes(Schema referenceSchema) {
+    super(referenceSchema);
+  }
+
+  static Schema fixup(Schema schema, Schema referenceSchema) {
+    return new Schema(TypeUtil.visit(schema,
+        new FlinkFixupTypes(referenceSchema)).asStructType().fields());
+  }
+
+  @Override
+  protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) {
+    if (type instanceof Types.FixedType) {
+      int length = ((Types.FixedType) type).length();
+      return source.typeId() == Type.TypeID.UUID && length == 16;
+    }
+    return false;
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
new file mode 100644
index 0000000..0827b21
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
@@ -0,0 +1,175 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+/**
+ * Converter between Flink types and Iceberg type.
+ * The conversion is not a 1:1 mapping that not allows back-and-forth conversion. So some information might get lost
+ * during the back-and-forth conversion.
+ * <p>
+ * This inconsistent types:
+ * <ul>
+ *   <li>map Iceberg UUID type to Flink BinaryType(16)</li>
+ *   <li>map Flink VarCharType(_) and CharType(_) to Iceberg String type</li>
+ *   <li>map Flink VarBinaryType(_) to Iceberg Binary type</li>
+ *   <li>map Flink TimeType(_) to Iceberg Time type (microseconds)</li>
+ *   <li>map Flink TimestampType(_) to Iceberg Timestamp without zone type (microseconds)</li>
+ *   <li>map Flink LocalZonedTimestampType(_) to Iceberg Timestamp with zone type (microseconds)</li>
+ *   <li>map Flink MultiSetType to Iceberg Map type(element, int)</li>
+ * </ul>
+ * <p>
+ */
+public class FlinkSchemaUtil {
+
+  private FlinkSchemaUtil() {
+  }
+
+  /**
+   * Convert the flink table schema to apache iceberg schema.
+   */
+  public static Schema convert(TableSchema schema) {
+    LogicalType schemaType = schema.toRowDataType().getLogicalType();
+    Preconditions.checkArgument(schemaType instanceof RowType, "Schema logical type should be RowType.");
+
+    RowType root = (RowType) schemaType;
+    Type converted = root.accept(new FlinkTypeToType(root));
+
+    Schema iSchema = new Schema(converted.asStructType().fields());
+    return freshIdentifierFieldIds(iSchema, schema);
+  }
+
+  private static Schema freshIdentifierFieldIds(Schema iSchema, TableSchema schema) {
+    // Locate the identifier field id list.
+    Set<Integer> identifierFieldIds = Sets.newHashSet();
+    if (schema.getPrimaryKey().isPresent()) {
+      for (String column : schema.getPrimaryKey().get().getColumns()) {
+        Types.NestedField field = iSchema.findField(column);
+        Preconditions.checkNotNull(field,
+            "Cannot find field ID for the primary key column %s in schema %s", column, iSchema);
+        identifierFieldIds.add(field.fieldId());
+      }
+    }
+
+    return new Schema(iSchema.schemaId(), iSchema.asStruct().fields(), identifierFieldIds);
+  }
+
+  /**
+   * Convert a Flink {@link TableSchema} to a {@link Schema} based on the given schema.
+   * <p>
+   * This conversion does not assign new ids; it uses ids from the base schema.
+   * <p>
+   * Data types, field order, and nullability will match the Flink type. This conversion may return
+   * a schema that is not compatible with base schema.
+   *
+   * @param baseSchema a Schema on which conversion is based
+   * @param flinkSchema a Flink TableSchema
+   * @return the equivalent Schema
+   * @throws IllegalArgumentException if the type cannot be converted or there are missing ids
+   */
+  public static Schema convert(Schema baseSchema, TableSchema flinkSchema) {
+    // convert to a type with fresh ids
+    Types.StructType struct = convert(flinkSchema).asStruct();
+    // reassign ids to match the base schema
+    Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema);
+    // fix types that can't be represented in Flink (UUID)
+    Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema);
+    return freshIdentifierFieldIds(fixedSchema, flinkSchema);
+  }
+
+  /**
+   * Convert a {@link Schema} to a {@link RowType Flink type}.
+   *
+   * @param schema a Schema
+   * @return the equivalent Flink type
+   * @throws IllegalArgumentException if the type cannot be converted to Flink
+   */
+  public static RowType convert(Schema schema) {
+    return (RowType) TypeUtil.visit(schema, new TypeToFlinkType());
+  }
+
+  /**
+   * Convert a {@link Type} to a {@link LogicalType Flink type}.
+   *
+   * @param type a Type
+   * @return the equivalent Flink type
+   * @throws IllegalArgumentException if the type cannot be converted to Flink
+   */
+  public static LogicalType convert(Type type) {
+    return TypeUtil.visit(type, new TypeToFlinkType());
+  }
+
+  /**
+   * Convert a {@link RowType} to a {@link TableSchema}.
+   *
+   * @param rowType a RowType
+   * @return Flink TableSchema
+   */
+  public static TableSchema toSchema(RowType rowType) {
+    TableSchema.Builder builder = TableSchema.builder();
+    for (RowType.RowField field : rowType.getFields()) {
+      builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType()));
+    }
+    return builder.build();
+  }
+
+  /**
+   * Convert a {@link Schema} to a {@link TableSchema}.
+   *
+   * @param schema iceberg schema to convert.
+   * @return Flink TableSchema.
+   */
+  public static TableSchema toSchema(Schema schema) {
+    TableSchema.Builder builder = TableSchema.builder();
+
+    // Add columns.
+    for (RowType.RowField field : convert(schema).getFields()) {
+      builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType()));
+    }
+
+    // Add primary key.
+    Set<Integer> identifierFieldIds = schema.identifierFieldIds();
+    if (!identifierFieldIds.isEmpty()) {
+      List<String> columns = Lists.newArrayListWithExpectedSize(identifierFieldIds.size());
+      for (Integer identifierFieldId : identifierFieldIds) {
+        String columnName = schema.findColumnName(identifierFieldId);
+        Preconditions.checkNotNull(columnName, "Cannot find field with id %s in schema %s", identifierFieldId, schema);
+
+        columns.add(columnName);
+      }
+      builder.primaryKey(columns.toArray(new String[0]));
+    }
+
+    return builder.build();
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java
new file mode 100644
index 0000000..88276d8
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java
@@ -0,0 +1,199 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.MultisetType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+
+class FlinkTypeToType extends FlinkTypeVisitor<Type> {
+
+  private final RowType root;
+  private int nextId;
+
+  FlinkTypeToType(RowType root) {
+    this.root = root;
+    // the root struct's fields use the first ids
+    this.nextId = root.getFieldCount();
+  }
+
+  private int getNextId() {
+    int next = nextId;
+    nextId += 1;
+    return next;
+  }
+
+  @Override
+  public Type visit(CharType charType) {
+    return Types.StringType.get();
+  }
+
+  @Override
+  public Type visit(VarCharType varCharType) {
+    return Types.StringType.get();
+  }
+
+  @Override
+  public Type visit(BooleanType booleanType) {
+    return Types.BooleanType.get();
+  }
+
+  @Override
+  public Type visit(BinaryType binaryType) {
+    return Types.FixedType.ofLength(binaryType.getLength());
+  }
+
+  @Override
+  public Type visit(VarBinaryType varBinaryType) {
+    return Types.BinaryType.get();
+  }
+
+  @Override
+  public Type visit(DecimalType decimalType) {
+    return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale());
+  }
+
+  @Override
+  public Type visit(TinyIntType tinyIntType) {
+    return Types.IntegerType.get();
+  }
+
+  @Override
+  public Type visit(SmallIntType smallIntType) {
+    return Types.IntegerType.get();
+  }
+
+  @Override
+  public Type visit(IntType intType) {
+    return Types.IntegerType.get();
+  }
+
+  @Override
+  public Type visit(BigIntType bigIntType) {
+    return Types.LongType.get();
+  }
+
+  @Override
+  public Type visit(FloatType floatType) {
+    return Types.FloatType.get();
+  }
+
+  @Override
+  public Type visit(DoubleType doubleType) {
+    return Types.DoubleType.get();
+  }
+
+  @Override
+  public Type visit(DateType dateType) {
+    return Types.DateType.get();
+  }
+
+  @Override
+  public Type visit(TimeType timeType) {
+    return Types.TimeType.get();
+  }
+
+  @Override
+  public Type visit(TimestampType timestampType) {
+    return Types.TimestampType.withoutZone();
+  }
+
+  @Override
+  public Type visit(LocalZonedTimestampType localZonedTimestampType) {
+    return Types.TimestampType.withZone();
+  }
+
+  @Override
+  public Type visit(ArrayType arrayType) {
+    Type elementType = arrayType.getElementType().accept(this);
+    if (arrayType.getElementType().isNullable()) {
+      return Types.ListType.ofOptional(getNextId(), elementType);
+    } else {
+      return Types.ListType.ofRequired(getNextId(), elementType);
+    }
+  }
+
+  @Override
+  public Type visit(MultisetType multisetType) {
+    Type elementType = multisetType.getElementType().accept(this);
+    return Types.MapType.ofRequired(getNextId(), getNextId(), elementType, Types.IntegerType.get());
+  }
+
+  @Override
+  public Type visit(MapType mapType) {
+    // keys in map are not allowed to be null.
+    Type keyType = mapType.getKeyType().accept(this);
+    Type valueType = mapType.getValueType().accept(this);
+    if (mapType.getValueType().isNullable()) {
+      return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType);
+    } else {
+      return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType);
+    }
+  }
+
+  @Override
+  @SuppressWarnings("ReferenceEquality")
+  public Type visit(RowType rowType) {
+    List<Types.NestedField> newFields = Lists.newArrayListWithExpectedSize(rowType.getFieldCount());
+    boolean isRoot = root == rowType;
+
+    List<Type> types = rowType.getFields().stream()
+        .map(f -> f.getType().accept(this))
+        .collect(Collectors.toList());
+
+    for (int i = 0; i < rowType.getFieldCount(); i++) {
+      int id = isRoot ? i : getNextId();
+
+      RowType.RowField field = rowType.getFields().get(i);
+      String name = field.getName();
+      String comment = field.getDescription().orElse(null);
+
+      if (field.getType().isNullable()) {
+        newFields.add(Types.NestedField.optional(id, name, types.get(i), comment));
+      } else {
+        newFields.add(Types.NestedField.required(id, name, types.get(i), comment));
+      }
+    }
+
+    return Types.StructType.of(newFields);
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
new file mode 100644
index 0000000..9d1a3c4
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+import org.apache.flink.table.types.logical.DayTimeIntervalType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeVisitor;
+import org.apache.flink.table.types.logical.NullType;
+import org.apache.flink.table.types.logical.RawType;
+import org.apache.flink.table.types.logical.StructuredType;
+import org.apache.flink.table.types.logical.SymbolType;
+import org.apache.flink.table.types.logical.YearMonthIntervalType;
+import org.apache.flink.table.types.logical.ZonedTimestampType;
+
+public abstract class FlinkTypeVisitor<T> implements LogicalTypeVisitor<T> {
+
+  // ------------------------- Unsupported types ------------------------------
+
+  @Override
+  public T visit(ZonedTimestampType zonedTimestampType) {
+    throw new UnsupportedOperationException("Unsupported ZonedTimestampType.");
+  }
+
+  @Override
+  public T visit(YearMonthIntervalType yearMonthIntervalType) {
+    throw new UnsupportedOperationException("Unsupported YearMonthIntervalType.");
+  }
+
+  @Override
+  public T visit(DayTimeIntervalType dayTimeIntervalType) {
+    throw new UnsupportedOperationException("Unsupported DayTimeIntervalType.");
+  }
+
+  @Override
+  public T visit(DistinctType distinctType) {
+    throw new UnsupportedOperationException("Unsupported DistinctType.");
+  }
+
+  @Override
+  public T visit(StructuredType structuredType) {
+    throw new UnsupportedOperationException("Unsupported StructuredType.");
+  }
+
+  @Override
+  public T visit(NullType nullType) {
+    throw new UnsupportedOperationException("Unsupported NullType.");
+  }
+
+  @Override
+  public T visit(RawType<?> rawType) {
+    throw new UnsupportedOperationException("Unsupported RawType.");
+  }
+
+  @Override
+  public T visit(SymbolType<?> symbolType) {
+    throw new UnsupportedOperationException("Unsupported SymbolType.");
+  }
+
+  @Override
+  public T visit(LogicalType other) {
+    throw new UnsupportedOperationException("Unsupported type: " + other);
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
new file mode 100644
index 0000000..93cf5ad
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
@@ -0,0 +1,98 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite;
+import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.flink.sink.FlinkSink;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite {
+  private final TableLoader tableLoader;
+  private final TableSchema tableSchema;
+
+  private boolean overwrite = false;
+
+  private IcebergTableSink(IcebergTableSink toCopy) {
+    this.tableLoader = toCopy.tableLoader;
+    this.tableSchema = toCopy.tableSchema;
+    this.overwrite = toCopy.overwrite;
+  }
+
+  public IcebergTableSink(TableLoader tableLoader, TableSchema tableSchema) {
+    this.tableLoader = tableLoader;
+    this.tableSchema = tableSchema;
+  }
+
+  @Override
+  public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+    Preconditions.checkState(!overwrite || context.isBounded(),
+        "Unbounded data stream doesn't support overwrite operation.");
+
+    List<String> equalityColumns = tableSchema.getPrimaryKey()
+        .map(UniqueConstraint::getColumns)
+        .orElseGet(ImmutableList::of);
+
+    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
+        .tableLoader(tableLoader)
+        .tableSchema(tableSchema)
+        .equalityFieldColumns(equalityColumns)
+        .overwrite(overwrite)
+        .append();
+  }
+
+  @Override
+  public void applyStaticPartition(Map<String, String> partition) {
+    // The flink's PartitionFanoutWriter will handle the static partition write policy automatically.
+  }
+
+  @Override
+  public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+    ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+    for (RowKind kind : requestedMode.getContainedKinds()) {
+      builder.addContainedKind(kind);
+    }
+    return builder.build();
+  }
+
+  @Override
+  public DynamicTableSink copy() {
+    return new IcebergTableSink(this);
+  }
+
+  @Override
+  public String asSummaryString() {
+    return "Iceberg table sink";
+  }
+
+  @Override
+  public void applyOverwrite(boolean newOverwrite) {
+    this.overwrite = newOverwrite;
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
new file mode 100644
index 0000000..dd8f645
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
@@ -0,0 +1,182 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DataStreamScanProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown;
+import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
+import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.flink.source.FlinkSource;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * Flink Iceberg table source.
+ */
+public class IcebergTableSource
+    implements ScanTableSource, SupportsProjectionPushDown, SupportsFilterPushDown, SupportsLimitPushDown {
+
+  private int[] projectedFields;
+  private long limit;
+  private List<Expression> filters;
+
+  private final TableLoader loader;
+  private final TableSchema schema;
+  private final Map<String, String> properties;
+  private final boolean isLimitPushDown;
+  private final ReadableConfig readableConfig;
+
+  private IcebergTableSource(IcebergTableSource toCopy) {
+    this.loader = toCopy.loader;
+    this.schema = toCopy.schema;
+    this.properties = toCopy.properties;
+    this.projectedFields = toCopy.projectedFields;
+    this.isLimitPushDown = toCopy.isLimitPushDown;
+    this.limit = toCopy.limit;
+    this.filters = toCopy.filters;
+    this.readableConfig = toCopy.readableConfig;
+  }
+
+  public IcebergTableSource(TableLoader loader, TableSchema schema, Map<String, String> properties,
+                            ReadableConfig readableConfig) {
+    this(loader, schema, properties, null, false, -1, ImmutableList.of(), readableConfig);
+  }
+
+  private IcebergTableSource(TableLoader loader, TableSchema schema, Map<String, String> properties,
+                             int[] projectedFields, boolean isLimitPushDown,
+                             long limit, List<Expression> filters, ReadableConfig readableConfig) {
+    this.loader = loader;
+    this.schema = schema;
+    this.properties = properties;
+    this.projectedFields = projectedFields;
+    this.isLimitPushDown = isLimitPushDown;
+    this.limit = limit;
+    this.filters = filters;
+    this.readableConfig = readableConfig;
+  }
+
+  @Override
+  public void applyProjection(int[][] projectFields) {
+    this.projectedFields = new int[projectFields.length];
+    for (int i = 0; i < projectFields.length; i++) {
+      Preconditions.checkArgument(projectFields[i].length == 1,
+          "Don't support nested projection in iceberg source now.");
+      this.projectedFields[i] = projectFields[i][0];
+    }
+  }
+
+  private DataStream<RowData> createDataStream(StreamExecutionEnvironment execEnv) {
+    return FlinkSource.forRowData()
+        .env(execEnv)
+        .tableLoader(loader)
+        .properties(properties)
+        .project(getProjectedSchema())
+        .limit(limit)
+        .filters(filters)
+        .flinkConf(readableConfig)
+        .build();
+  }
+
+  private TableSchema getProjectedSchema() {
+    if (projectedFields == null) {
+      return schema;
+    } else {
+      String[] fullNames = schema.getFieldNames();
+      DataType[] fullTypes = schema.getFieldDataTypes();
+      return TableSchema.builder().fields(
+          Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new),
+          Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)).build();
+    }
+  }
+
+  @Override
+  public void applyLimit(long newLimit) {
+    this.limit = newLimit;
+  }
+
+  @Override
+  public Result applyFilters(List<ResolvedExpression> flinkFilters) {
+    List<ResolvedExpression> acceptedFilters = Lists.newArrayList();
+    List<Expression> expressions = Lists.newArrayList();
+
+    for (ResolvedExpression resolvedExpression : flinkFilters) {
+      Optional<Expression> icebergExpression = FlinkFilters.convert(resolvedExpression);
+      if (icebergExpression.isPresent()) {
+        expressions.add(icebergExpression.get());
+        acceptedFilters.add(resolvedExpression);
+      }
+    }
+
+    this.filters = expressions;
+    return Result.of(acceptedFilters, flinkFilters);
+  }
+
+  @Override
+  public boolean supportsNestedProjection() {
+    // TODO: support nested projection
+    return false;
+  }
+
+  @Override
+  public ChangelogMode getChangelogMode() {
+    return ChangelogMode.insertOnly();
+  }
+
+  @Override
+  public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
+    return new DataStreamScanProvider() {
+      @Override
+      public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv) {
+        return createDataStream(execEnv);
+      }
+
+      @Override
+      public boolean isBounded() {
+        return FlinkSource.isBounded(properties);
+      }
+    };
+  }
+
+  @Override
+  public DynamicTableSource copy() {
+    return new IcebergTableSource(this);
+  }
+
+  @Override
+  public String asSummaryString() {
+    return "Iceberg table source";
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java
new file mode 100644
index 0000000..401e9db
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java
@@ -0,0 +1,138 @@
+/*
+ * 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;
+
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.time.LocalDateTime;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.UUIDUtil;
+
+public class RowDataWrapper implements StructLike {
+
+  private final LogicalType[] types;
+  private final PositionalGetter<?>[] getters;
+  private RowData rowData = null;
+
+  public RowDataWrapper(RowType rowType, Types.StructType struct) {
+    int size = rowType.getFieldCount();
+
+    types = (LogicalType[]) Array.newInstance(LogicalType.class, size);
+    getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size);
+
+    for (int i = 0; i < size; i++) {
+      types[i] = rowType.getTypeAt(i);
+      getters[i] = buildGetter(types[i], struct.fields().get(i).type());
+    }
+  }
+
+  public RowDataWrapper wrap(RowData data) {
+    this.rowData = data;
+    return this;
+  }
+
+  @Override
+  public int size() {
+    return types.length;
+  }
+
+  @Override
+  public <T> T get(int pos, Class<T> javaClass) {
+    if (rowData.isNullAt(pos)) {
+      return null;
+    } else if (getters[pos] != null) {
+      return javaClass.cast(getters[pos].get(rowData, pos));
+    }
+
+    Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData);
+    return javaClass.cast(value);
+  }
+
+  @Override
+  public <T> void set(int pos, T value) {
+    throw new UnsupportedOperationException("Could not set a field in the RowDataWrapper because rowData is read-only");
+  }
+
+  private interface PositionalGetter<T> {
+    T get(RowData data, int pos);
+  }
+
+  private static PositionalGetter<?> buildGetter(LogicalType logicalType, Type type) {
+    switch (logicalType.getTypeRoot()) {
+      case TINYINT:
+        return (row, pos) -> (int) row.getByte(pos);
+      case SMALLINT:
+        return (row, pos) -> (int) row.getShort(pos);
+      case CHAR:
+      case VARCHAR:
+        return (row, pos) -> row.getString(pos).toString();
+
+      case BINARY:
+      case VARBINARY:
+        if (Type.TypeID.UUID == type.typeId()) {
+          return (row, pos) -> UUIDUtil.convert(row.getBinary(pos));
+        } else {
+          return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos));
+        }
+
+      case DECIMAL:
+        DecimalType decimalType = (DecimalType) logicalType;
+        return (row, pos) -> row.getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()).toBigDecimal();
+
+      case TIME_WITHOUT_TIME_ZONE:
+        // Time in RowData is in milliseconds (Integer), while iceberg's time is microseconds (Long).
+        return (row, pos) -> ((long) row.getInt(pos)) * 1_000;
+
+      case TIMESTAMP_WITHOUT_TIME_ZONE:
+        TimestampType timestampType = (TimestampType) logicalType;
+        return (row, pos) -> {
+          LocalDateTime localDateTime = row.getTimestamp(pos, timestampType.getPrecision()).toLocalDateTime();
+          return DateTimeUtil.microsFromTimestamp(localDateTime);
+        };
+
+      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+        LocalZonedTimestampType lzTs = (LocalZonedTimestampType) logicalType;
+        return (row, pos) -> {
+          TimestampData timestampData = row.getTimestamp(pos, lzTs.getPrecision());
+          return timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000;
+        };
+
+      case ROW:
+        RowType rowType = (RowType) logicalType;
+        Types.StructType structType = (Types.StructType) type;
+
+        RowDataWrapper nestedWrapper = new RowDataWrapper(rowType, structType);
+        return (row, pos) -> nestedWrapper.wrap(row.getRow(pos, rowType.getFieldCount()));
+
+      default:
+        return null;
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java
new file mode 100644
index 0000000..ebcb1fb
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java
@@ -0,0 +1,131 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+/**
+ * Serializable loader to load an Iceberg {@link Table}.
+ * Flink needs to get {@link Table} objects in the cluster (for example, to get splits), not just on the client side.
+ * So we need an Iceberg table loader to get the {@link Table} object.
+ */
+public interface TableLoader extends Closeable, Serializable {
+
+  void open();
+
+  Table loadTable();
+
+  static TableLoader fromCatalog(CatalogLoader catalogLoader, TableIdentifier identifier) {
+    return new CatalogTableLoader(catalogLoader, identifier);
+  }
+
+  static TableLoader fromHadoopTable(String location) {
+    return fromHadoopTable(location, FlinkCatalogFactory.clusterHadoopConf());
+  }
+
+  static TableLoader fromHadoopTable(String location, Configuration hadoopConf) {
+    return new HadoopTableLoader(location, hadoopConf);
+  }
+
+  class HadoopTableLoader implements TableLoader {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String location;
+    private final SerializableConfiguration hadoopConf;
+
+    private transient HadoopTables tables;
+
+    private HadoopTableLoader(String location, Configuration conf) {
+      this.location = location;
+      this.hadoopConf = new SerializableConfiguration(conf);
+    }
+
+    @Override
+    public void open() {
+      tables = new HadoopTables(hadoopConf.get());
+    }
+
+    @Override
+    public Table loadTable() {
+      return tables.load(location);
+    }
+
+    @Override
+    public void close() {
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+          .add("location", location)
+          .toString();
+    }
+  }
+
+  class CatalogTableLoader implements TableLoader {
+
+    private static final long serialVersionUID = 1L;
+
+    private final CatalogLoader catalogLoader;
+    private final String identifier;
+
+    private transient Catalog catalog;
+
+    private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) {
+      this.catalogLoader = catalogLoader;
+      this.identifier = tableIdentifier.toString();
+    }
+
+    @Override
+    public void open() {
+      catalog = catalogLoader.loadCatalog();
+    }
+
+    @Override
+    public Table loadTable() {
+      return catalog.loadTable(TableIdentifier.parse(identifier));
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (catalog instanceof Closeable) {
+        ((Closeable) catalog).close();
+      }
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+          .add("tableIdentifier", identifier)
+          .add("catalogLoader", catalogLoader)
+          .toString();
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java
new file mode 100644
index 0000000..cf594b3
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java
@@ -0,0 +1,134 @@
+/*
+ * 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;
+
+import java.util.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+class TypeToFlinkType extends TypeUtil.SchemaVisitor<LogicalType> {
+  TypeToFlinkType() {
+  }
+
+  @Override
+  public LogicalType schema(Schema schema, LogicalType structType) {
+    return structType;
+  }
+
+  @Override
+  public LogicalType struct(Types.StructType struct, List<LogicalType> fieldResults) {
+    List<Types.NestedField> fields = struct.fields();
+
+    List<RowType.RowField> flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size());
+    for (int i = 0; i < fields.size(); i += 1) {
+      Types.NestedField field = fields.get(i);
+      LogicalType type = fieldResults.get(i);
+      RowType.RowField flinkField = new RowType.RowField(
+          field.name(), type.copy(field.isOptional()), field.doc());
+      flinkFields.add(flinkField);
+    }
+
+    return new RowType(flinkFields);
+  }
+
+  @Override
+  public LogicalType field(Types.NestedField field, LogicalType fieldResult) {
+    return fieldResult;
+  }
+
+  @Override
+  public LogicalType list(Types.ListType list, LogicalType elementResult) {
+    return new ArrayType(elementResult.copy(list.isElementOptional()));
+  }
+
+  @Override
+  public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) {
+    // keys in map are not allowed to be null.
+    return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional()));
+  }
+
+  @Override
+  public LogicalType primitive(Type.PrimitiveType primitive) {
+    switch (primitive.typeId()) {
+      case BOOLEAN:
+        return new BooleanType();
+      case INTEGER:
+        return new IntType();
+      case LONG:
+        return new BigIntType();
+      case FLOAT:
+        return new FloatType();
+      case DOUBLE:
+        return new DoubleType();
+      case DATE:
+        return new DateType();
+      case TIME:
+        // For the type: Flink only support TimeType with default precision (second) now. The precision of time is
+        // not supported in Flink, so we can think of it as a simple time type directly.
+        // For the data: Flink uses int that support mills to represent time data, so it supports mills precision.
+        return new TimeType();
+      case TIMESTAMP:
+        Types.TimestampType timestamp = (Types.TimestampType) primitive;
+        if (timestamp.shouldAdjustToUTC()) {
+          // MICROS
+          return new LocalZonedTimestampType(6);
+        } else {
+          // MICROS
+          return new TimestampType(6);
+        }
+      case STRING:
+        return new VarCharType(VarCharType.MAX_LENGTH);
+      case UUID:
+        // UUID length is 16
+        return new BinaryType(16);
+      case FIXED:
+        Types.FixedType fixedType = (Types.FixedType) primitive;
+        return new BinaryType(fixedType.length());
+      case BINARY:
+        return new VarBinaryType(VarBinaryType.MAX_LENGTH);
+      case DECIMAL:
+        Types.DecimalType decimal = (Types.DecimalType) primitive;
+        return new DecimalType(decimal.precision(), decimal.scale());
+      default:
+        throw new UnsupportedOperationException(
+            "Cannot convert unknown type to Flink: " + primitive);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
new file mode 100644
index 0000000..98702ce
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
@@ -0,0 +1,53 @@
+/*
+ * 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.actions;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.iceberg.Table;
+
+public class Actions {
+
+  public static final Configuration CONFIG = new Configuration()
+      // disable classloader check as Avro may cache class/object in the serializers.
+      .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false);
+
+  private StreamExecutionEnvironment env;
+  private Table table;
+
+  private Actions(StreamExecutionEnvironment env, Table table) {
+    this.env = env;
+    this.table = table;
+  }
+
+  public static Actions forTable(StreamExecutionEnvironment env, Table table) {
+    return new Actions(env, table);
+  }
+
+  public static Actions forTable(Table table) {
+    return new Actions(StreamExecutionEnvironment.getExecutionEnvironment(CONFIG), table);
+  }
+
+  public RewriteDataFilesAction rewriteDataFiles() {
+    return new RewriteDataFilesAction(env, table);
+  }
+
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java
new file mode 100644
index 0000000..291be0c
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java
@@ -0,0 +1,72 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BaseRewriteDataFilesAction;
+import org.apache.iceberg.flink.source.RowDataRewriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class RewriteDataFilesAction extends BaseRewriteDataFilesAction<RewriteDataFilesAction> {
+
+  private StreamExecutionEnvironment env;
+  private int maxParallelism;
+
+  public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) {
+    super(table);
+    this.env = env;
+    this.maxParallelism = env.getParallelism();
+  }
+
+  @Override
+  protected FileIO fileIO() {
+    return table().io();
+  }
+
+  @Override
+  protected List<DataFile> rewriteDataForTasks(List<CombinedScanTask> combinedScanTasks) {
+    int size = combinedScanTasks.size();
+    int parallelism = Math.min(size, maxParallelism);
+    DataStream<CombinedScanTask> dataStream = env.fromCollection(combinedScanTasks);
+    RowDataRewriter rowDataRewriter = new RowDataRewriter(table(), caseSensitive(), fileIO(), encryptionManager());
+    try {
+      return rowDataRewriter.rewriteDataForTasks(dataStream, parallelism);
+    } catch (Exception e) {
+      throw new RuntimeException("Rewrite data file error.", e);
+    }
+  }
+
+  @Override
+  protected RewriteDataFilesAction self() {
+    return this;
+  }
+
+  public RewriteDataFilesAction maxParallelism(int parallelism) {
+    Preconditions.checkArgument(parallelism > 0, "Invalid max parallelism %d", parallelism);
+    this.maxParallelism = parallelism;
+    return this;
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java
new file mode 100644
index 0000000..1ccc3b7
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java
@@ -0,0 +1,73 @@
+/*
+ * 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.data;
+
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.NullType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.avro.AvroWithPartnerByStructureVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.Pair;
+
+public abstract class AvroWithFlinkSchemaVisitor<T> extends AvroWithPartnerByStructureVisitor<LogicalType, T> {
+
+  @Override
+  protected boolean isStringType(LogicalType logicalType) {
+    return logicalType.getTypeRoot().getFamilies().contains(LogicalTypeFamily.CHARACTER_STRING);
+  }
+
+  @Override
+  protected boolean isMapType(LogicalType logicalType) {
+    return logicalType instanceof MapType;
+  }
+
+  @Override
+  protected LogicalType arrayElementType(LogicalType arrayType) {
+    Preconditions.checkArgument(arrayType instanceof ArrayType, "Invalid array: %s is not an array", arrayType);
+    return ((ArrayType) arrayType).getElementType();
+  }
+
+  @Override
+  protected LogicalType mapKeyType(LogicalType mapType) {
+    Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType);
+    return ((MapType) mapType).getKeyType();
+  }
+
+  @Override
+  protected LogicalType mapValueType(LogicalType mapType) {
+    Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType);
+    return ((MapType) mapType).getValueType();
+  }
+
+  @Override
+  protected Pair<String, LogicalType> fieldNameAndType(LogicalType structType, int pos) {
+    Preconditions.checkArgument(structType instanceof RowType, "Invalid struct: %s is not a struct", structType);
+    RowType.RowField field = ((RowType) structType).getFields().get(pos);
+    return Pair.of(field.getName(), field.getType());
+  }
+
+  @Override
+  protected LogicalType nullType() {
+    return new NullType();
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java
new file mode 100644
index 0000000..991ef63
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java
@@ -0,0 +1,167 @@
+/*
+ * 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.data;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.Decoder;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor;
+import org.apache.iceberg.avro.SupportsRowPosition;
+import org.apache.iceberg.avro.ValueReader;
+import org.apache.iceberg.avro.ValueReaders;
+import org.apache.iceberg.data.avro.DecoderResolver;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+
+public class FlinkAvroReader implements DatumReader<RowData>, SupportsRowPosition {
+
+  private final Schema readSchema;
+  private final ValueReader<RowData> reader;
+  private Schema fileSchema = null;
+
+  public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) {
+    this(expectedSchema, readSchema, ImmutableMap.of());
+  }
+
+  @SuppressWarnings("unchecked")
+  public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map<Integer, ?> constants) {
+    this.readSchema = readSchema;
+    this.reader = (ValueReader<RowData>) AvroSchemaWithTypeVisitor
+        .visit(expectedSchema, readSchema, new ReadBuilder(constants));
+  }
+
+  @Override
+  public void setSchema(Schema newFileSchema) {
+    this.fileSchema = Schema.applyAliases(newFileSchema, readSchema);
+  }
+
+  @Override
+  public RowData read(RowData reuse, Decoder decoder) throws IOException {
+    return DecoderResolver.resolveAndRead(decoder, readSchema, fileSchema, reader, reuse);
+  }
+
+  @Override
+  public void setRowPositionSupplier(Supplier<Long> posSupplier) {
+    if (reader instanceof SupportsRowPosition) {
+      ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier);
+    }
+  }
+
+  private static class ReadBuilder extends AvroSchemaWithTypeVisitor<ValueReader<?>> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ValueReader<?> record(Types.StructType expected, Schema record, List<String> names,
+                                 List<ValueReader<?>> fields) {
+      return FlinkValueReaders.struct(fields, expected.asStructType(), idToConstant);
+    }
+
+    @Override
+    public ValueReader<?> union(Type expected, Schema union, List<ValueReader<?>> options) {
+      return ValueReaders.union(options);
+    }
+
+    @Override
+    public ValueReader<?> array(Types.ListType expected, Schema array, ValueReader<?> elementReader) {
+      return FlinkValueReaders.array(elementReader);
+    }
+
+    @Override
+    public ValueReader<?> map(Types.MapType expected, Schema map,
+                              ValueReader<?> keyReader, ValueReader<?> valueReader) {
+      return FlinkValueReaders.arrayMap(keyReader, valueReader);
+    }
+
+    @Override
+    public ValueReader<?> map(Types.MapType expected, Schema map, ValueReader<?> valueReader) {
+      return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader);
+    }
+
+    @Override
+    public ValueReader<?> primitive(Type.PrimitiveType expected, Schema primitive) {
+      LogicalType logicalType = primitive.getLogicalType();
+      if (logicalType != null) {
+        switch (logicalType.getName()) {
+          case "date":
+            return ValueReaders.ints();
+
+          case "time-micros":
+            return FlinkValueReaders.timeMicros();
+
+          case "timestamp-millis":
+            return FlinkValueReaders.timestampMills();
+
+          case "timestamp-micros":
+            return FlinkValueReaders.timestampMicros();
+
+          case "decimal":
+            LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType;
+            return FlinkValueReaders.decimal(
+                ValueReaders.decimalBytesReader(primitive),
+                decimal.getPrecision(),
+                decimal.getScale());
+
+          case "uuid":
+            return FlinkValueReaders.uuids();
+
+          default:
+            throw new IllegalArgumentException("Unknown logical type: " + logicalType);
+        }
+      }
+
+      switch (primitive.getType()) {
+        case NULL:
+          return ValueReaders.nulls();
+        case BOOLEAN:
+          return ValueReaders.booleans();
+        case INT:
+          return ValueReaders.ints();
+        case LONG:
+          return ValueReaders.longs();
+        case FLOAT:
+          return ValueReaders.floats();
+        case DOUBLE:
+          return ValueReaders.doubles();
+        case STRING:
+          return FlinkValueReaders.strings();
+        case FIXED:
+          return ValueReaders.fixed(primitive.getFixedSize());
+        case BYTES:
+          return ValueReaders.bytes();
+        case ENUM:
+          return FlinkValueReaders.enums(primitive.getEnumSymbols());
+        default:
+          throw new IllegalArgumentException("Unsupported type: " + primitive);
+      }
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java
new file mode 100644
index 0000000..b069a35
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java
@@ -0,0 +1,156 @@
+/*
+ * 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.data;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.io.Encoder;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.FieldMetrics;
+import org.apache.iceberg.avro.MetricsAwareDatumWriter;
+import org.apache.iceberg.avro.ValueWriter;
+import org.apache.iceberg.avro.ValueWriters;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FlinkAvroWriter implements MetricsAwareDatumWriter<RowData> {
+  private final RowType rowType;
+  private ValueWriter<RowData> writer = null;
+
+  public FlinkAvroWriter(RowType rowType) {
+    this.rowType = rowType;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void setSchema(Schema schema) {
+    this.writer = (ValueWriter<RowData>) AvroWithFlinkSchemaVisitor
+        .visit(rowType, schema, new WriteBuilder());
+  }
+
+  @Override
+  public void write(RowData datum, Encoder out) throws IOException {
+    writer.write(datum, out);
+  }
+
+  @Override
+  public Stream<FieldMetrics> metrics() {
+    return writer.metrics();
+  }
+
+  private static class WriteBuilder extends AvroWithFlinkSchemaVisitor<ValueWriter<?>> {
+    @Override
+    public ValueWriter<?> record(LogicalType struct, Schema record, List<String> names, List<ValueWriter<?>> fields) {
+      return FlinkValueWriters.row(fields, IntStream.range(0, names.size())
+          .mapToObj(i -> fieldNameAndType(struct, i).second()).collect(Collectors.toList()));
+    }
+
+    @Override
+    public ValueWriter<?> union(LogicalType type, Schema union, List<ValueWriter<?>> options) {
+      Preconditions.checkArgument(options.contains(ValueWriters.nulls()),
+          "Cannot create writer for non-option union: %s", union);
+      Preconditions.checkArgument(options.size() == 2,
+          "Cannot create writer for non-option union: %s", union);
+      if (union.getTypes().get(0).getType() == Schema.Type.NULL) {
+        return ValueWriters.option(0, options.get(1));
+      } else {
+        return ValueWriters.option(1, options.get(0));
+      }
+    }
+
+    @Override
+    public ValueWriter<?> array(LogicalType sArray, Schema array, ValueWriter<?> elementWriter) {
+      return FlinkValueWriters.array(elementWriter, arrayElementType(sArray));
+    }
+
+    @Override
+    public ValueWriter<?> map(LogicalType sMap, Schema map, ValueWriter<?> valueReader) {
+      return FlinkValueWriters.map(FlinkValueWriters.strings(), mapKeyType(sMap), valueReader, mapValueType(sMap));
+    }
+
+    @Override
+    public ValueWriter<?> map(LogicalType sMap, Schema map, ValueWriter<?> keyWriter, ValueWriter<?> valueWriter) {
+      return FlinkValueWriters.arrayMap(keyWriter, mapKeyType(sMap), valueWriter, mapValueType(sMap));
+    }
+
+    @Override
+    public ValueWriter<?> primitive(LogicalType type, Schema primitive) {
+      org.apache.avro.LogicalType logicalType = primitive.getLogicalType();
+      if (logicalType != null) {
+        switch (logicalType.getName()) {
+          case "date":
+            return ValueWriters.ints();
+
+          case "time-micros":
+            return FlinkValueWriters.timeMicros();
+
+          case "timestamp-micros":
+            return FlinkValueWriters.timestampMicros();
+
+          case "decimal":
+            LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType;
+            return FlinkValueWriters.decimal(decimal.getPrecision(), decimal.getScale());
+
+          case "uuid":
+            return ValueWriters.uuids();
+
+          default:
+            throw new IllegalArgumentException("Unsupported logical type: " + logicalType);
+        }
+      }
+
+      switch (primitive.getType()) {
+        case NULL:
+          return ValueWriters.nulls();
+        case BOOLEAN:
+          return ValueWriters.booleans();
+        case INT:
+          switch (type.getTypeRoot()) {
+            case TINYINT:
+              return ValueWriters.tinyints();
+            case SMALLINT:
+              return ValueWriters.shorts();
+            default:
+              return ValueWriters.ints();
+          }
+        case LONG:
+          return ValueWriters.longs();
+        case FLOAT:
+          return ValueWriters.floats();
+        case DOUBLE:
+          return ValueWriters.doubles();
+        case STRING:
+          return FlinkValueWriters.strings();
+        case FIXED:
+          return ValueWriters.fixed(primitive.getFixedSize());
+        case BYTES:
+          return ValueWriters.bytes();
+        default:
+          throw new IllegalArgumentException("Unsupported type: " + primitive);
+      }
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java
new file mode 100644
index 0000000..4c4e205
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java
@@ -0,0 +1,125 @@
+/*
+ * 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.data;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcRowReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.orc.OrcValueReader;
+import org.apache.iceberg.orc.OrcValueReaders;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class FlinkOrcReader implements OrcRowReader<RowData> {
+  private final OrcValueReader<?> reader;
+
+  public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) {
+    this(iSchema, readSchema, ImmutableMap.of());
+  }
+
+  public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map<Integer, ?> idToConstant) {
+    this.reader = OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant));
+  }
+
+  @Override
+  public RowData read(VectorizedRowBatch batch, int row) {
+    return (RowData) reader.read(new StructColumnVector(batch.size, batch.cols), row);
+  }
+
+  @Override
+  public void setBatchContext(long batchOffsetInFile) {
+    reader.setBatchContext(batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<OrcValueReader<?>> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public OrcValueReader<RowData> record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                                          List<OrcValueReader<?>> fields) {
+      return FlinkOrcReaders.struct(fields, iStruct, idToConstant);
+    }
+
+    @Override
+    public OrcValueReader<ArrayData> list(Types.ListType iList, TypeDescription array,
+                                          OrcValueReader<?> elementReader) {
+      return FlinkOrcReaders.array(elementReader);
+    }
+
+    @Override
+    public OrcValueReader<MapData> map(Types.MapType iMap, TypeDescription map,
+                                       OrcValueReader<?> keyReader,
+                                       OrcValueReader<?> valueReader) {
+      return FlinkOrcReaders.map(keyReader, valueReader);
+    }
+
+    @Override
+    public OrcValueReader<?> primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      switch (iPrimitive.typeId()) {
+        case BOOLEAN:
+          return OrcValueReaders.booleans();
+        case INTEGER:
+          return OrcValueReaders.ints();
+        case LONG:
+          return OrcValueReaders.longs();
+        case FLOAT:
+          return OrcValueReaders.floats();
+        case DOUBLE:
+          return OrcValueReaders.doubles();
+        case DATE:
+          return FlinkOrcReaders.dates();
+        case TIME:
+          return FlinkOrcReaders.times();
+        case TIMESTAMP:
+          Types.TimestampType timestampType = (Types.TimestampType) iPrimitive;
+          if (timestampType.shouldAdjustToUTC()) {
+            return FlinkOrcReaders.timestampTzs();
+          } else {
+            return FlinkOrcReaders.timestamps();
+          }
+        case STRING:
+          return FlinkOrcReaders.strings();
+        case UUID:
+        case FIXED:
+        case BINARY:
+          return OrcValueReaders.bytes();
+        case DECIMAL:
+          Types.DecimalType decimalType = (Types.DecimalType) iPrimitive;
+          return FlinkOrcReaders.decimals(decimalType.precision(), decimalType.scale());
+        default:
+          throw new IllegalArgumentException(String.format("Invalid iceberg type %s corresponding to ORC type %s",
+              iPrimitive, primitive));
+      }
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
new file mode 100644
index 0000000..744a05e
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
@@ -0,0 +1,272 @@
+/*
+ * 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.data;
+
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.orc.OrcValueReader;
+import org.apache.iceberg.orc.OrcValueReaders;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
+
+class FlinkOrcReaders {
+  private FlinkOrcReaders() {
+  }
+
+  static OrcValueReader<StringData> strings() {
+    return StringReader.INSTANCE;
+  }
+
+  static OrcValueReader<Integer> dates() {
+    return DateReader.INSTANCE;
+  }
+
+  static OrcValueReader<DecimalData> decimals(int precision, int scale) {
+    if (precision <= 18) {
+      return new Decimal18Reader(precision, scale);
+    } else if (precision <= 38) {
+      return new Decimal38Reader(precision, scale);
+    } else {
+      throw new IllegalArgumentException("Invalid precision: " + precision);
+    }
+  }
+
+  static OrcValueReader<Integer> times() {
+    return TimeReader.INSTANCE;
+  }
+
+  static OrcValueReader<TimestampData> timestamps() {
+    return TimestampReader.INSTANCE;
+  }
+
+  static OrcValueReader<TimestampData> timestampTzs() {
+    return TimestampTzReader.INSTANCE;
+  }
+
+  static <T> OrcValueReader<ArrayData> array(OrcValueReader<T> elementReader) {
+    return new ArrayReader<>(elementReader);
+  }
+
+  public static <K, V> OrcValueReader<MapData> map(OrcValueReader<K> keyReader, OrcValueReader<V> valueReader) {
+    return new MapReader<>(keyReader, valueReader);
+  }
+
+  public static OrcValueReader<RowData> struct(List<OrcValueReader<?>> readers,
+                                               Types.StructType struct,
+                                               Map<Integer, ?> idToConstant) {
+    return new StructReader(readers, struct, idToConstant);
+  }
+
+  private static class StringReader implements OrcValueReader<StringData> {
+    private static final StringReader INSTANCE = new StringReader();
+
+    @Override
+    public StringData nonNullRead(ColumnVector vector, int row) {
+      BytesColumnVector bytesVector = (BytesColumnVector) vector;
+      return StringData.fromBytes(bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]);
+    }
+  }
+
+  private static class DateReader implements OrcValueReader<Integer> {
+    private static final DateReader INSTANCE = new DateReader();
+
+    @Override
+    public Integer nonNullRead(ColumnVector vector, int row) {
+      return (int) ((LongColumnVector) vector).vector[row];
+    }
+  }
+
+  private static class Decimal18Reader implements OrcValueReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    Decimal18Reader(int precision, int scale) {
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData nonNullRead(ColumnVector vector, int row) {
+      HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row];
+
+      // The hive ORC writer may will adjust the scale of decimal data.
+      Preconditions.checkArgument(value.precision() <= precision,
+          "Cannot read value as decimal(%s,%s), too large: %s", precision, scale, value);
+
+      return DecimalData.fromUnscaledLong(value.serialize64(scale), precision, scale);
+    }
+  }
+
+  private static class Decimal38Reader implements OrcValueReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    Decimal38Reader(int precision, int scale) {
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData nonNullRead(ColumnVector vector, int row) {
+      BigDecimal value = ((DecimalColumnVector) vector).vector[row].getHiveDecimal().bigDecimalValue();
+
+      Preconditions.checkArgument(value.precision() <= precision,
+          "Cannot read value as decimal(%s,%s), too large: %s", precision, scale, value);
+
+      return DecimalData.fromBigDecimal(value, precision, scale);
+    }
+  }
+
+  private static class TimeReader implements OrcValueReader<Integer> {
+    private static final TimeReader INSTANCE = new TimeReader();
+
+    @Override
+    public Integer nonNullRead(ColumnVector vector, int row) {
+      long micros = ((LongColumnVector) vector).vector[row];
+      // Flink only support time mills, just erase micros.
+      return (int) (micros / 1000);
+    }
+  }
+
+  private static class TimestampReader implements OrcValueReader<TimestampData> {
+    private static final TimestampReader INSTANCE = new TimestampReader();
+
+    @Override
+    public TimestampData nonNullRead(ColumnVector vector, int row) {
+      TimestampColumnVector tcv = (TimestampColumnVector) vector;
+      LocalDateTime localDate = Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row])
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime();
+      return TimestampData.fromLocalDateTime(localDate);
+    }
+  }
+
+  private static class TimestampTzReader implements OrcValueReader<TimestampData> {
+    private static final TimestampTzReader INSTANCE = new TimestampTzReader();
+
+    @Override
+    public TimestampData nonNullRead(ColumnVector vector, int row) {
+      TimestampColumnVector tcv = (TimestampColumnVector) vector;
+      Instant instant = Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row])
+          .atOffset(ZoneOffset.UTC)
+          .toInstant();
+      return TimestampData.fromInstant(instant);
+    }
+  }
+
+  private static class ArrayReader<T> implements OrcValueReader<ArrayData> {
+    private final OrcValueReader<T> elementReader;
+
+    private ArrayReader(OrcValueReader<T> elementReader) {
+      this.elementReader = elementReader;
+    }
+
+    @Override
+    public ArrayData nonNullRead(ColumnVector vector, int row) {
+      ListColumnVector listVector = (ListColumnVector) vector;
+      int offset = (int) listVector.offsets[row];
+      int length = (int) listVector.lengths[row];
+      List<T> elements = Lists.newArrayListWithExpectedSize(length);
+      for (int c = 0; c < length; ++c) {
+        elements.add(elementReader.read(listVector.child, offset + c));
+      }
+      return new GenericArrayData(elements.toArray());
+    }
+
+    @Override
+    public void setBatchContext(long batchOffsetInFile) {
+      elementReader.setBatchContext(batchOffsetInFile);
+    }
+  }
+
+  private static class MapReader<K, V> implements OrcValueReader<MapData> {
+    private final OrcValueReader<K> keyReader;
+    private final OrcValueReader<V> valueReader;
+
+    private MapReader(OrcValueReader<K> keyReader, OrcValueReader<V> valueReader) {
+      this.keyReader = keyReader;
+      this.valueReader = valueReader;
+    }
+
+    @Override
+    public MapData nonNullRead(ColumnVector vector, int row) {
+      MapColumnVector mapVector = (MapColumnVector) vector;
+      int offset = (int) mapVector.offsets[row];
+      long length = mapVector.lengths[row];
+
+      Map<K, V> map = Maps.newHashMap();
+      for (int c = 0; c < length; c++) {
+        K key = keyReader.read(mapVector.keys, offset + c);
+        V value = valueReader.read(mapVector.values, offset + c);
+        map.put(key, value);
+      }
+
+      return new GenericMapData(map);
+    }
+
+    @Override
+    public void setBatchContext(long batchOffsetInFile) {
+      keyReader.setBatchContext(batchOffsetInFile);
+      valueReader.setBatchContext(batchOffsetInFile);
+    }
+  }
+
+  private static class StructReader extends OrcValueReaders.StructReader<RowData> {
+    private final int numFields;
+
+    StructReader(List<OrcValueReader<?>> readers, Types.StructType struct, Map<Integer, ?> idToConstant) {
+      super(readers, struct, idToConstant);
+      this.numFields = struct.fields().size();
+    }
+
+    @Override
+    protected RowData create() {
+      return new GenericRowData(numFields);
+    }
+
+    @Override
+    protected void set(RowData struct, int pos, Object value) {
+      ((GenericRowData) struct).setField(pos, value);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java
new file mode 100644
index 0000000..2eeb268
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java
@@ -0,0 +1,151 @@
+/*
+ * 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.data;
+
+import java.util.Deque;
+import java.util.List;
+import java.util.stream.Stream;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.FieldMetrics;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.orc.GenericOrcWriters;
+import org.apache.iceberg.orc.OrcRowWriter;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class FlinkOrcWriter implements OrcRowWriter<RowData> {
+  private final FlinkOrcWriters.RowDataWriter writer;
+
+  private FlinkOrcWriter(RowType rowType, Schema iSchema) {
+    this.writer = (FlinkOrcWriters.RowDataWriter) FlinkSchemaVisitor.visit(rowType, iSchema, new WriteBuilder());
+  }
+
+  public static OrcRowWriter<RowData> buildWriter(RowType rowType, Schema iSchema) {
+    return new FlinkOrcWriter(rowType, iSchema);
+  }
+
+  @Override
+  public void write(RowData row, VectorizedRowBatch output) {
+    Preconditions.checkArgument(row != null, "value must not be null");
+    writer.writeRow(row, output);
+  }
+
+  @Override
+  public List<OrcValueWriter<?>> writers() {
+    return writer.writers();
+  }
+
+  @Override
+  public Stream<FieldMetrics<?>> metrics() {
+    return writer.metrics();
+  }
+
+  private static class WriteBuilder extends FlinkSchemaVisitor<OrcValueWriter<?>> {
+    private final Deque<Integer> fieldIds = Lists.newLinkedList();
+
+    private WriteBuilder() {
+    }
+
+    @Override
+    public void beforeField(Types.NestedField field) {
+      fieldIds.push(field.fieldId());
+    }
+
+    @Override
+    public void afterField(Types.NestedField field) {
+      fieldIds.pop();
+    }
+
+    @Override
+    public OrcValueWriter<RowData> record(Types.StructType iStruct,
+                                          List<OrcValueWriter<?>> results,
+                                          List<LogicalType> fieldType) {
+      return FlinkOrcWriters.struct(results, fieldType);
+    }
+
+    @Override
+    public OrcValueWriter<?> map(Types.MapType iMap, OrcValueWriter<?> key, OrcValueWriter<?> value,
+                                 LogicalType keyType, LogicalType valueType) {
+      return FlinkOrcWriters.map(key, value, keyType, valueType);
+    }
+
+    @Override
+    public OrcValueWriter<?> list(Types.ListType iList, OrcValueWriter<?> element, LogicalType elementType) {
+      return FlinkOrcWriters.list(element, elementType);
+    }
+
+    @Override
+    public OrcValueWriter<?> primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) {
+      switch (iPrimitive.typeId()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case INTEGER:
+          switch (flinkPrimitive.getTypeRoot()) {
+            case TINYINT:
+              return GenericOrcWriters.bytes();
+            case SMALLINT:
+              return GenericOrcWriters.shorts();
+          }
+          return GenericOrcWriters.ints();
+        case LONG:
+          return GenericOrcWriters.longs();
+        case FLOAT:
+          Preconditions.checkArgument(fieldIds.peek() != null,
+              String.format("[BUG] Cannot find field id for primitive field with type %s. This is likely because id " +
+                  "information is not properly pushed during schema visiting.", iPrimitive));
+          return GenericOrcWriters.floats(fieldIds.peek());
+        case DOUBLE:
+          Preconditions.checkArgument(fieldIds.peek() != null,
+              String.format("[BUG] Cannot find field id for primitive field with type %s. This is likely because id " +
+              "information is not properly pushed during schema visiting.", iPrimitive));
+          return GenericOrcWriters.doubles(fieldIds.peek());
+        case DATE:
+          return FlinkOrcWriters.dates();
+        case TIME:
+          return FlinkOrcWriters.times();
+        case TIMESTAMP:
+          Types.TimestampType timestampType = (Types.TimestampType) iPrimitive;
+          if (timestampType.shouldAdjustToUTC()) {
+            return FlinkOrcWriters.timestampTzs();
+          } else {
+            return FlinkOrcWriters.timestamps();
+          }
+        case STRING:
+          return FlinkOrcWriters.strings();
+        case UUID:
+        case FIXED:
+        case BINARY:
+          return GenericOrcWriters.byteArrays();
+        case DECIMAL:
+          Types.DecimalType decimalType = (Types.DecimalType) iPrimitive;
+          return FlinkOrcWriters.decimals(decimalType.precision(), decimalType.scale());
+        default:
+          throw new IllegalArgumentException(String.format(
+              "Invalid iceberg type %s corresponding to Flink logical type %s", iPrimitive, flinkPrimitive));
+      }
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java
new file mode 100644
index 0000000..6b596ac
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java
@@ -0,0 +1,294 @@
+/*
+ * 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.data;
+
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.stream.Stream;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.iceberg.FieldMetrics;
+import org.apache.iceberg.data.orc.GenericOrcWriters;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.orc.storage.common.type.HiveDecimal;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+
+class FlinkOrcWriters {
+
+  private FlinkOrcWriters() {
+  }
+
+  static OrcValueWriter<StringData> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  static OrcValueWriter<Integer> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  static OrcValueWriter<Integer> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  static OrcValueWriter<TimestampData> timestamps() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  static OrcValueWriter<TimestampData> timestampTzs() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  static OrcValueWriter<DecimalData> decimals(int precision, int scale) {
+    if (precision <= 18) {
+      return new Decimal18Writer(precision, scale);
+    } else if (precision <= 38) {
+      return new Decimal38Writer(precision, scale);
+    } else {
+      throw new IllegalArgumentException("Invalid precision: " + precision);
+    }
+  }
+
+  static <T> OrcValueWriter<ArrayData> list(OrcValueWriter<T> elementWriter, LogicalType elementType) {
+    return new ListWriter<>(elementWriter, elementType);
+  }
+
+  static <K, V> OrcValueWriter<MapData> map(OrcValueWriter<K> keyWriter, OrcValueWriter<V> valueWriter,
+                                            LogicalType keyType, LogicalType valueType) {
+    return new MapWriter<>(keyWriter, valueWriter, keyType, valueType);
+  }
+
+  static OrcValueWriter<RowData> struct(List<OrcValueWriter<?>> writers, List<LogicalType> types) {
+    return new RowDataWriter(writers, types);
+  }
+
+  private static class StringWriter implements OrcValueWriter<StringData> {
+    private static final StringWriter INSTANCE = new StringWriter();
+
+    @Override
+    public void nonNullWrite(int rowId, StringData data, ColumnVector output) {
+      byte[] value = data.toBytes();
+      ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+    }
+  }
+
+  private static class DateWriter implements OrcValueWriter<Integer> {
+    private static final DateWriter INSTANCE = new DateWriter();
+
+    @Override
+    public void nonNullWrite(int rowId, Integer data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class TimeWriter implements OrcValueWriter<Integer> {
+    private static final TimeWriter INSTANCE = new TimeWriter();
+
+    @Override
+    public void nonNullWrite(int rowId, Integer millis, ColumnVector output) {
+      // The time in flink is in millisecond, while the standard time in iceberg is microsecond.
+      // So we need to transform it to microsecond.
+      ((LongColumnVector) output).vector[rowId] = millis * 1000L;
+    }
+  }
+
+  private static class TimestampWriter implements OrcValueWriter<TimestampData> {
+    private static final TimestampWriter INSTANCE = new TimestampWriter();
+
+    @Override
+    public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.setIsUTC(true);
+      // millis
+      OffsetDateTime offsetDateTime = data.toInstant().atOffset(ZoneOffset.UTC);
+      cv.time[rowId] = offsetDateTime.toEpochSecond() * 1_000 + offsetDateTime.getNano() / 1_000_000;
+      // truncate nanos to only keep microsecond precision.
+      cv.nanos[rowId] = (offsetDateTime.getNano() / 1_000) * 1_000;
+    }
+  }
+
+  private static class TimestampTzWriter implements OrcValueWriter<TimestampData> {
+    private static final TimestampTzWriter INSTANCE = new TimestampTzWriter();
+
+    @Override
+    public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      // millis
+      Instant instant = data.toInstant();
+      cv.time[rowId] = instant.toEpochMilli();
+      // truncate nanos to only keep microsecond precision.
+      cv.nanos[rowId] = (instant.getNano() / 1_000) * 1_000;
+    }
+  }
+
+  private static class Decimal18Writer implements OrcValueWriter<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    Decimal18Writer(int precision, int scale) {
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) {
+      Preconditions.checkArgument(scale == data.scale(),
+          "Cannot write value as decimal(%s,%s), wrong scale: %s", precision, scale, data);
+      Preconditions.checkArgument(data.precision() <= precision,
+          "Cannot write value as decimal(%s,%s), too large: %s", precision, scale, data);
+
+      ((DecimalColumnVector) output).vector[rowId].setFromLongAndScale(data.toUnscaledLong(), data.scale());
+    }
+  }
+
+  private static class Decimal38Writer implements OrcValueWriter<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    Decimal38Writer(int precision, int scale) {
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) {
+      Preconditions.checkArgument(scale == data.scale(),
+          "Cannot write value as decimal(%s,%s), wrong scale: %s", precision, scale, data);
+      Preconditions.checkArgument(data.precision() <= precision,
+          "Cannot write value as decimal(%s,%s), too large: %s", precision, scale, data);
+
+      ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data.toBigDecimal(), false));
+    }
+  }
+
+  static class ListWriter<T> implements OrcValueWriter<ArrayData> {
+    private final OrcValueWriter<T> elementWriter;
+    private final ArrayData.ElementGetter elementGetter;
+
+    ListWriter(OrcValueWriter<T> elementWriter, LogicalType elementType) {
+      this.elementWriter = elementWriter;
+      this.elementGetter = ArrayData.createElementGetter(elementType);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void nonNullWrite(int rowId, ArrayData data, ColumnVector output) {
+      ListColumnVector cv = (ListColumnVector) output;
+      cv.lengths[rowId] = data.size();
+      cv.offsets[rowId] = cv.childCount;
+      cv.childCount = (int) (cv.childCount + cv.lengths[rowId]);
+      // make sure the child is big enough.
+      growColumnVector(cv.child, cv.childCount);
+
+      for (int e = 0; e < cv.lengths[rowId]; ++e) {
+        Object value = elementGetter.getElementOrNull(data, e);
+        elementWriter.write((int) (e + cv.offsets[rowId]), (T) value, cv.child);
+      }
+    }
+
+    @Override
+    public Stream<FieldMetrics<?>> metrics() {
+      return elementWriter.metrics();
+    }
+
+  }
+
+  static class MapWriter<K, V> implements OrcValueWriter<MapData> {
+    private final OrcValueWriter<K> keyWriter;
+    private final OrcValueWriter<V> valueWriter;
+    private final ArrayData.ElementGetter keyGetter;
+    private final ArrayData.ElementGetter valueGetter;
+
+    MapWriter(OrcValueWriter<K> keyWriter, OrcValueWriter<V> valueWriter,
+              LogicalType keyType, LogicalType valueType) {
+      this.keyWriter = keyWriter;
+      this.valueWriter = valueWriter;
+      this.keyGetter = ArrayData.createElementGetter(keyType);
+      this.valueGetter = ArrayData.createElementGetter(valueType);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void nonNullWrite(int rowId, MapData data, ColumnVector output) {
+      MapColumnVector cv = (MapColumnVector) output;
+      ArrayData keyArray = data.keyArray();
+      ArrayData valArray = data.valueArray();
+
+      // record the length and start of the list elements
+      cv.lengths[rowId] = data.size();
+      cv.offsets[rowId] = cv.childCount;
+      cv.childCount = (int) (cv.childCount + cv.lengths[rowId]);
+      // make sure the child is big enough
+      growColumnVector(cv.keys, cv.childCount);
+      growColumnVector(cv.values, cv.childCount);
+      // Add each element
+      for (int e = 0; e < cv.lengths[rowId]; ++e) {
+        int pos = (int) (e + cv.offsets[rowId]);
+        keyWriter.write(pos, (K) keyGetter.getElementOrNull(keyArray, e), cv.keys);
+        valueWriter.write(pos, (V) valueGetter.getElementOrNull(valArray, e), cv.values);
+      }
+    }
+
+    @Override
+    public Stream<FieldMetrics<?>> metrics() {
+      return Stream.concat(keyWriter.metrics(), valueWriter.metrics());
+    }
+  }
+
+  static class RowDataWriter extends GenericOrcWriters.StructWriter<RowData> {
+    private final List<RowData.FieldGetter> fieldGetters;
+
+    RowDataWriter(List<OrcValueWriter<?>> writers, List<LogicalType> types) {
+      super(writers);
+
+      this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size());
+      for (int i = 0; i < types.size(); i++) {
+        fieldGetters.add(RowData.createFieldGetter(types.get(i), i));
+      }
+    }
+
+    @Override
+    protected Object get(RowData struct, int index) {
+      return fieldGetters.get(index).getFieldOrNull(struct);
+    }
+
+  }
+
+  private static void growColumnVector(ColumnVector cv, int requestedSize) {
+    if (cv.isNull.length < requestedSize) {
+      // Use growth factor of 3 to avoid frequent array allocations
+      cv.ensureSize(requestedSize * 3, true);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
new file mode 100644
index 0000000..d7088b4
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java
@@ -0,0 +1,799 @@
+/*
+ * 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.data;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.parquet.ParquetValueReader;
+import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ArrayUtil;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+public class FlinkParquetReaders {
+  private FlinkParquetReaders() {
+  }
+
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema, MessageType fileSchema) {
+    return buildReader(expectedSchema, fileSchema, ImmutableMap.of());
+  }
+
+  @SuppressWarnings("unchecked")
+  public static ParquetValueReader<RowData> buildReader(Schema expectedSchema,
+                                                        MessageType fileSchema,
+                                                        Map<Integer, ?> idToConstant) {
+    return (ParquetValueReader<RowData>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
+        new ReadBuilder(fileSchema, idToConstant)
+    );
+  }
+
+  private static class ReadBuilder extends TypeWithSchemaVisitor<ParquetValueReader<?>> {
+    private final MessageType type;
+    private final Map<Integer, ?> idToConstant;
+
+    ReadBuilder(MessageType type, Map<Integer, ?> idToConstant) {
+      this.type = type;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ParquetValueReader<RowData> message(Types.StructType expected, MessageType message,
+                                               List<ParquetValueReader<?>> fieldReaders) {
+      return struct(expected, message.asGroupType(), fieldReaders);
+    }
+
+    @Override
+    public ParquetValueReader<RowData> struct(Types.StructType expected, GroupType struct,
+                                              List<ParquetValueReader<?>> fieldReaders) {
+      // match the expected struct's order
+      Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
+      Map<Integer, Type> typesById = Maps.newHashMap();
+      List<Type> fields = struct.getFields();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = fields.get(i);
+        if (fieldReaders.get(i) != null) {
+          int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
+          if (fieldType.getId() != null) {
+            int id = fieldType.getId().intValue();
+            readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i)));
+            typesById.put(id, fieldType);
+          }
+        }
+      }
+
+      List<Types.NestedField> expectedFields = expected != null ?
+          expected.fields() : ImmutableList.of();
+      List<ParquetValueReader<?>> reorderedFields = Lists.newArrayListWithExpectedSize(
+          expectedFields.size());
+      List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
+      for (Types.NestedField field : expectedFields) {
+        int id = field.fieldId();
+        if (idToConstant.containsKey(id)) {
+          // containsKey is used because the constant may be null
+          reorderedFields.add(ParquetValueReaders.constant(idToConstant.get(id)));
+          types.add(null);
+        } else if (id == MetadataColumns.ROW_POSITION.fieldId()) {
+          reorderedFields.add(ParquetValueReaders.position());
+          types.add(null);
+        } else if (id == MetadataColumns.IS_DELETED.fieldId()) {
+          reorderedFields.add(ParquetValueReaders.constant(false));
+          types.add(null);
+        } else {
+          ParquetValueReader<?> reader = readersById.get(id);
+          if (reader != null) {
+            reorderedFields.add(reader);
+            types.add(typesById.get(id));
+          } else {
+            reorderedFields.add(ParquetValueReaders.nulls());
+            types.add(null);
+          }
+        }
+      }
+
+      return new RowDataReader(types, reorderedFields);
+    }
+
+    @Override
+    public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array,
+                                      ParquetValueReader<?> elementReader) {
+      if (expectedList == null) {
+        return null;
+      }
+
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1;
+
+      return new ArrayReader<>(repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader));
+    }
+
+    @Override
+    public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
+                                     ParquetValueReader<?> keyReader,
+                                     ParquetValueReader<?> valueReader) {
+      if (expectedMap == null) {
+        return null;
+      }
+
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
+
+      Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
+      Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
+
+      return new MapReader<>(repeatedD, repeatedR,
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
+    }
+
+    @Override
+    @SuppressWarnings("CyclomaticComplexity")
+    public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
+                                           PrimitiveType primitive) {
+      if (expected == null) {
+        return null;
+      }
+
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return new StringReader(desc);
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            if (expected.typeId() == Types.LongType.get().typeId()) {
+              return new ParquetValueReaders.IntAsLongReader(desc);
+            } else {
+              return new ParquetValueReaders.UnboxedReader<>(desc);
+            }
+          case TIME_MICROS:
+            return new LossyMicrosToMillisTimeReader(desc);
+          case TIME_MILLIS:
+            return new MillisTimeReader(desc);
+          case DATE:
+          case INT_64:
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          case TIMESTAMP_MICROS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new MicrosToTimestampTzReader(desc);
+            } else {
+              return new MicrosToTimestampReader(desc);
+            }
+          case TIMESTAMP_MILLIS:
+            if (((Types.TimestampType) expected).shouldAdjustToUTC()) {
+              return new MillisToTimestampTzReader(desc);
+            } else {
+              return new MillisToTimestampReader(desc);
+            }
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT64:
+                return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              case INT32:
+                return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return new ParquetValueReaders.ByteArrayReader(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new ParquetValueReaders.ByteArrayReader(desc);
+        case INT32:
+          if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) {
+            return new ParquetValueReaders.IntAsLongReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case FLOAT:
+          if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) {
+            return new ParquetValueReaders.FloatAsDoubleReader(desc);
+          } else {
+            return new ParquetValueReaders.UnboxedReader<>(desc);
+          }
+        case BOOLEAN:
+        case INT64:
+        case DOUBLE:
+          return new ParquetValueReaders.UnboxedReader<>(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static class BinaryDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      Binary binary = column.nextBinary();
+      BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale);
+      // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader
+      return DecimalData.fromBigDecimal(bigDecimal, precision, scale);
+    }
+  }
+
+  private static class IntegerDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale);
+    }
+  }
+
+  private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    LongDecimalReader(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(DecimalData ignored) {
+      return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale);
+    }
+  }
+
+  private static class MicrosToTimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    MicrosToTimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class MicrosToTimestampReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    MicrosToTimestampReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long value = readLong();
+      return TimestampData.fromInstant(Instant.ofEpochSecond(Math.floorDiv(value, 1000_000),
+          Math.floorMod(value, 1000_000) * 1000));
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class MillisToTimestampReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    MillisToTimestampReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long millis = readLong();
+      return TimestampData.fromEpochMillis(millis);
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class MillisToTimestampTzReader extends ParquetValueReaders.UnboxedReader<TimestampData> {
+    MillisToTimestampTzReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public TimestampData read(TimestampData ignored) {
+      long millis = readLong();
+      return TimestampData.fromLocalDateTime(Instant.ofEpochMilli(millis)
+          .atOffset(ZoneOffset.UTC)
+          .toLocalDateTime());
+    }
+
+    @Override
+    public long readLong() {
+      return column.nextLong();
+    }
+  }
+
+  private static class StringReader extends ParquetValueReaders.PrimitiveReader<StringData> {
+    StringReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public StringData read(StringData ignored) {
+      Binary binary = column.nextBinary();
+      ByteBuffer buffer = binary.toByteBuffer();
+      if (buffer.hasArray()) {
+        return StringData.fromBytes(
+            buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+      } else {
+        return StringData.fromBytes(binary.getBytes());
+      }
+    }
+  }
+
+  private static class LossyMicrosToMillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    LossyMicrosToMillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      // Discard microseconds since Flink uses millisecond unit for TIME type.
+      return (int) Math.floorDiv(column.nextLong(), 1000);
+    }
+  }
+
+  private static class MillisTimeReader extends ParquetValueReaders.PrimitiveReader<Integer> {
+    MillisTimeReader(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public Integer read(Integer reuse) {
+      return (int) column.nextLong();
+    }
+  }
+
+  private static class ArrayReader<E> extends ParquetValueReaders.RepeatedReader<ArrayData, ReusableArrayData, E> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader<E> reader) {
+      super(definitionLevel, repetitionLevel, reader);
+    }
+
+    @Override
+    protected ReusableArrayData newListData(ArrayData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableArrayData) {
+        return (ReusableArrayData) reuse;
+      } else {
+        return new ReusableArrayData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected E getElement(ReusableArrayData list) {
+      E value = null;
+      if (readPos < list.capacity()) {
+        value = (E) list.values[readPos];
+      }
+
+      readPos += 1;
+
+      return value;
+    }
+
+    @Override
+    protected void addElement(ReusableArrayData reused, E element) {
+      if (writePos >= reused.capacity()) {
+        reused.grow();
+      }
+
+      reused.values[writePos] = element;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected ArrayData buildList(ReusableArrayData list) {
+      list.setNumElements(writePos);
+      return list;
+    }
+  }
+
+  private static class MapReader<K, V> extends
+      ParquetValueReaders.RepeatedKeyValueReader<MapData, ReusableMapData, K, V> {
+    private int readPos = 0;
+    private int writePos = 0;
+
+    private final ParquetValueReaders.ReusableEntry<K, V> entry = new ParquetValueReaders.ReusableEntry<>();
+    private final ParquetValueReaders.ReusableEntry<K, V> nullEntry = new ParquetValueReaders.ReusableEntry<>();
+
+    MapReader(int definitionLevel, int repetitionLevel,
+              ParquetValueReader<K> keyReader, ParquetValueReader<V> valueReader) {
+      super(definitionLevel, repetitionLevel, keyReader, valueReader);
+    }
+
+    @Override
+    protected ReusableMapData newMapData(MapData reuse) {
+      this.readPos = 0;
+      this.writePos = 0;
+
+      if (reuse instanceof ReusableMapData) {
+        return (ReusableMapData) reuse;
+      } else {
+        return new ReusableMapData();
+      }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected Map.Entry<K, V> getPair(ReusableMapData map) {
+      Map.Entry<K, V> kv = nullEntry;
+      if (readPos < map.capacity()) {
+        entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]);
+        kv = entry;
+      }
+
+      readPos += 1;
+
+      return kv;
+    }
+
+    @Override
+    protected void addPair(ReusableMapData map, K key, V value) {
+      if (writePos >= map.capacity()) {
+        map.grow();
+      }
+
+      map.keys.values[writePos] = key;
+      map.values.values[writePos] = value;
+
+      writePos += 1;
+    }
+
+    @Override
+    protected MapData buildMap(ReusableMapData map) {
+      map.setNumElements(writePos);
+      return map;
+    }
+  }
+
+  private static class RowDataReader extends ParquetValueReaders.StructReader<RowData, GenericRowData> {
+    private final int numFields;
+
+    RowDataReader(List<Type> types, List<ParquetValueReader<?>> readers) {
+      super(types, readers);
+      this.numFields = readers.size();
+    }
+
+    @Override
+    protected GenericRowData newStructData(RowData reuse) {
+      if (reuse instanceof GenericRowData) {
+        return (GenericRowData) reuse;
+      } else {
+        return new GenericRowData(numFields);
+      }
+    }
+
+    @Override
+    protected Object getField(GenericRowData intermediate, int pos) {
+      return intermediate.getField(pos);
+    }
+
+    @Override
+    protected RowData buildStruct(GenericRowData struct) {
+      return struct;
+    }
+
+    @Override
+    protected void set(GenericRowData row, int pos, Object value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setNull(GenericRowData row, int pos) {
+      row.setField(pos, null);
+    }
+
+    @Override
+    protected void setBoolean(GenericRowData row, int pos, boolean value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setInteger(GenericRowData row, int pos, int value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setLong(GenericRowData row, int pos, long value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setFloat(GenericRowData row, int pos, float value) {
+      row.setField(pos, value);
+    }
+
+    @Override
+    protected void setDouble(GenericRowData row, int pos, double value) {
+      row.setField(pos, value);
+    }
+  }
+
+  private static class ReusableMapData implements MapData {
+    private final ReusableArrayData keys;
+    private final ReusableArrayData values;
+
+    private int numElements;
+
+    private ReusableMapData() {
+      this.keys = new ReusableArrayData();
+      this.values = new ReusableArrayData();
+    }
+
+    private void grow() {
+      keys.grow();
+      values.grow();
+    }
+
+    private int capacity() {
+      return keys.capacity();
+    }
+
+    public void setNumElements(int numElements) {
+      this.numElements = numElements;
+      keys.setNumElements(numElements);
+      values.setNumElements(numElements);
+    }
+
+    @Override
+    public int size() {
+      return numElements;
+    }
+
+    @Override
+    public ReusableArrayData keyArray() {
+      return keys;
+    }
+
+    @Override
+    public ReusableArrayData valueArray() {
+      return values;
+    }
+  }
+
+  private static class ReusableArrayData implements ArrayData {
+    private static final Object[] EMPTY = new Object[0];
+
+    private Object[] values = EMPTY;
+    private int numElements = 0;
+
+    private void grow() {
+      if (values.length == 0) {
+        this.values = new Object[20];
+      } else {
+        Object[] old = values;
+        this.values = new Object[old.length << 1];
+        // copy the old array in case it has values that can be reused
+        System.arraycopy(old, 0, values, 0, old.length);
+      }
+    }
+
+    private int capacity() {
+      return values.length;
+    }
+
+    public void setNumElements(int numElements) {
+      this.numElements = numElements;
+    }
+
+    @Override
+    public int size() {
+      return numElements;
+    }
+
+    @Override
+    public boolean isNullAt(int ordinal) {
+      return null == values[ordinal];
+    }
+
+    @Override
+    public boolean getBoolean(int ordinal) {
+      return (boolean) values[ordinal];
+    }
+
+    @Override
+    public byte getByte(int ordinal) {
+      return (byte) values[ordinal];
+    }
+
+    @Override
+    public short getShort(int ordinal) {
+      return (short) values[ordinal];
+    }
+
+    @Override
+    public int getInt(int ordinal) {
+      return (int) values[ordinal];
+    }
+
+    @Override
+    public long getLong(int ordinal) {
+      return (long) values[ordinal];
+    }
+
+    @Override
+    public float getFloat(int ordinal) {
+      return (float) values[ordinal];
+    }
+
+    @Override
+    public double getDouble(int ordinal) {
+      return (double) values[ordinal];
+    }
+
+    @Override
+    public StringData getString(int pos) {
+      return (StringData) values[pos];
+    }
+
+    @Override
+    public DecimalData getDecimal(int pos, int precision, int scale) {
+      return (DecimalData) values[pos];
+    }
+
+    @Override
+    public TimestampData getTimestamp(int pos, int precision) {
+      return (TimestampData) values[pos];
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> RawValueData<T> getRawValue(int pos) {
+      return (RawValueData<T>) values[pos];
+    }
+
+    @Override
+    public byte[] getBinary(int ordinal) {
+      return (byte[]) values[ordinal];
+    }
+
+    @Override
+    public ArrayData getArray(int ordinal) {
+      return (ArrayData) values[ordinal];
+    }
+
+    @Override
+    public MapData getMap(int ordinal) {
+      return (MapData) values[ordinal];
+    }
+
+    @Override
+    public RowData getRow(int pos, int numFields) {
+      return (RowData) values[pos];
+    }
+
+    @Override
+    public boolean[] toBooleanArray() {
+      return ArrayUtil.toPrimitive((Boolean[]) values);
+    }
+
+    @Override
+    public byte[] toByteArray() {
+      return ArrayUtil.toPrimitive((Byte[]) values);
+    }
+
+    @Override
+    public short[] toShortArray() {
+      return ArrayUtil.toPrimitive((Short[]) values);
+    }
+
+    @Override
+    public int[] toIntArray() {
+      return ArrayUtil.toPrimitive((Integer[]) values);
+    }
+
+    @Override
+    public long[] toLongArray() {
+      return ArrayUtil.toPrimitive((Long[]) values);
+    }
+
+    @Override
+    public float[] toFloatArray() {
+      return ArrayUtil.toPrimitive((Float[]) values);
+    }
+
+    @Override
+    public double[] toDoubleArray() {
+      return ArrayUtil.toPrimitive((Double[]) values);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java
new file mode 100644
index 0000000..6154ef1
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java
@@ -0,0 +1,456 @@
+/*
+ * 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.data;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.RowType.RowField;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.iceberg.parquet.ParquetValueReaders;
+import org.apache.iceberg.parquet.ParquetValueWriter;
+import org.apache.iceberg.parquet.ParquetValueWriters;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.DecimalUtil;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+public class FlinkParquetWriters {
+  private FlinkParquetWriters() {
+  }
+
+  @SuppressWarnings("unchecked")
+  public static <T> ParquetValueWriter<T> buildWriter(LogicalType schema, MessageType type) {
+    return (ParquetValueWriter<T>) ParquetWithFlinkSchemaVisitor.visit(schema, type, new WriteBuilder(type));
+  }
+
+  private static class WriteBuilder extends ParquetWithFlinkSchemaVisitor<ParquetValueWriter<?>> {
+    private final MessageType type;
+
+    WriteBuilder(MessageType type) {
+      this.type = type;
+    }
+
+    @Override
+    public ParquetValueWriter<?> message(RowType sStruct, MessageType message, List<ParquetValueWriter<?>> fields) {
+      return struct(sStruct, message.asGroupType(), fields);
+    }
+
+    @Override
+    public ParquetValueWriter<?> struct(RowType sStruct, GroupType struct,
+                                        List<ParquetValueWriter<?>> fieldWriters) {
+      List<Type> fields = struct.getFields();
+      List<RowField> flinkFields = sStruct.getFields();
+      List<ParquetValueWriter<?>> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size());
+      List<LogicalType> flinkTypes = Lists.newArrayList();
+      for (int i = 0; i < fields.size(); i += 1) {
+        writers.add(newOption(struct.getType(i), fieldWriters.get(i)));
+        flinkTypes.add(flinkFields.get(i).getType());
+      }
+
+      return new RowDataWriter(writers, flinkTypes);
+    }
+
+    @Override
+    public ParquetValueWriter<?> list(ArrayType sArray, GroupType array, ParquetValueWriter<?> elementWriter) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath);
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath);
+
+      return new ArrayDataWriter<>(repeatedD, repeatedR,
+          newOption(repeated.getType(0), elementWriter),
+          sArray.getElementType());
+    }
+
+    @Override
+    public ParquetValueWriter<?> map(MapType sMap, GroupType map,
+                                     ParquetValueWriter<?> keyWriter, ParquetValueWriter<?> valueWriter) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath);
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath);
+
+      return new MapDataWriter<>(repeatedD, repeatedR,
+          newOption(repeatedKeyValue.getType(0), keyWriter),
+          newOption(repeatedKeyValue.getType(1), valueWriter),
+          sMap.getKeyType(), sMap.getValueType());
+    }
+
+
+    private ParquetValueWriter<?> newOption(Type fieldType, ParquetValueWriter<?> writer) {
+      int maxD = type.getMaxDefinitionLevel(path(fieldType.getName()));
+      return ParquetValueWriters.option(fieldType, maxD, writer);
+    }
+
+    @Override
+    public ParquetValueWriter<?> primitive(LogicalType fType, PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return strings(desc);
+          case DATE:
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            return ints(fType, desc);
+          case INT_64:
+            return ParquetValueWriters.longs(desc);
+          case TIME_MICROS:
+            return timeMicros(desc);
+          case TIMESTAMP_MICROS:
+            return timestamps(desc);
+          case DECIMAL:
+            DecimalLogicalTypeAnnotation decimal = (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation();
+            switch (primitive.getPrimitiveTypeName()) {
+              case INT32:
+                return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale());
+              case INT64:
+                return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale());
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return byteArrays(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return byteArrays(desc);
+        case BOOLEAN:
+          return ParquetValueWriters.booleans(desc);
+        case INT32:
+          return ints(fType, desc);
+        case INT64:
+          return ParquetValueWriters.longs(desc);
+        case FLOAT:
+          return ParquetValueWriters.floats(desc);
+        case DOUBLE:
+          return ParquetValueWriters.doubles(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  private static ParquetValueWriters.PrimitiveWriter<?> ints(LogicalType type, ColumnDescriptor desc) {
+    if (type instanceof TinyIntType) {
+      return ParquetValueWriters.tinyints(desc);
+    } else if (type instanceof SmallIntType) {
+      return ParquetValueWriters.shorts(desc);
+    }
+    return ParquetValueWriters.ints(desc);
+  }
+
+  private static ParquetValueWriters.PrimitiveWriter<StringData> strings(ColumnDescriptor desc) {
+    return new StringDataWriter(desc);
+  }
+
+  private static ParquetValueWriters.PrimitiveWriter<Integer> timeMicros(ColumnDescriptor desc) {
+    return new TimeMicrosWriter(desc);
+  }
+
+  private static ParquetValueWriters.PrimitiveWriter<DecimalData> decimalAsInteger(ColumnDescriptor desc,
+                                                                                   int precision, int scale) {
+    Preconditions.checkArgument(precision <= 9, "Cannot write decimal value as integer with precision larger than 9," +
+        " wrong precision %s", precision);
+    return new IntegerDecimalWriter(desc, precision, scale);
+  }
+
+  private static ParquetValueWriters.PrimitiveWriter<DecimalData> decimalAsLong(ColumnDescriptor desc,
+                                                                                int precision, int scale) {
+    Preconditions.checkArgument(precision <= 18, "Cannot write decimal value as long with precision larger than 18, " +
+        " wrong precision %s", precision);
+    return new LongDecimalWriter(desc, precision, scale);
+  }
+
+  private static ParquetValueWriters.PrimitiveWriter<DecimalData> decimalAsFixed(ColumnDescriptor desc,
+                                                                                 int precision, int scale) {
+    return new FixedDecimalWriter(desc, precision, scale);
+  }
+
+  private static ParquetValueWriters.PrimitiveWriter<TimestampData> timestamps(ColumnDescriptor desc) {
+    return new TimestampDataWriter(desc);
+  }
+
+  private static ParquetValueWriters.PrimitiveWriter<byte[]> byteArrays(ColumnDescriptor desc) {
+    return new ByteArrayWriter(desc);
+  }
+
+  private static class StringDataWriter extends ParquetValueWriters.PrimitiveWriter<StringData> {
+    private StringDataWriter(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public void write(int repetitionLevel, StringData value) {
+      column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.toBytes()));
+    }
+  }
+
+  private static class TimeMicrosWriter extends ParquetValueWriters.PrimitiveWriter<Integer> {
+    private TimeMicrosWriter(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public void write(int repetitionLevel, Integer value) {
+      long micros = value.longValue() * 1000;
+      column.writeLong(repetitionLevel, micros);
+    }
+  }
+
+  private static class IntegerDecimalWriter extends ParquetValueWriters.PrimitiveWriter<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public void write(int repetitionLevel, DecimalData decimal) {
+      Preconditions.checkArgument(decimal.scale() == scale,
+          "Cannot write value as decimal(%s,%s), wrong scale: %s", precision, scale, decimal);
+      Preconditions.checkArgument(decimal.precision() <= precision,
+          "Cannot write value as decimal(%s,%s), too large: %s", precision, scale, decimal);
+
+      column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong());
+    }
+  }
+
+  private static class LongDecimalWriter extends ParquetValueWriters.PrimitiveWriter<DecimalData> {
+    private final int precision;
+    private final int scale;
+
+    private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public void write(int repetitionLevel, DecimalData decimal) {
+      Preconditions.checkArgument(decimal.scale() == scale,
+          "Cannot write value as decimal(%s,%s), wrong scale: %s", precision, scale, decimal);
+      Preconditions.checkArgument(decimal.precision() <= precision,
+          "Cannot write value as decimal(%s,%s), too large: %s", precision, scale, decimal);
+
+      column.writeLong(repetitionLevel, decimal.toUnscaledLong());
+    }
+  }
+
+  private static class FixedDecimalWriter extends ParquetValueWriters.PrimitiveWriter<DecimalData> {
+    private final int precision;
+    private final int scale;
+    private final ThreadLocal<byte[]> bytes;
+
+    private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+      this.bytes = ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]);
+    }
+
+    @Override
+    public void write(int repetitionLevel, DecimalData decimal) {
+      byte[] binary = DecimalUtil.toReusedFixLengthBytes(precision, scale, decimal.toBigDecimal(), bytes.get());
+      column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary));
+    }
+  }
+
+  private static class TimestampDataWriter extends ParquetValueWriters.PrimitiveWriter<TimestampData> {
+    private TimestampDataWriter(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public void write(int repetitionLevel, TimestampData value) {
+      column.writeLong(repetitionLevel, value.getMillisecond() * 1000 + value.getNanoOfMillisecond() / 1000);
+    }
+  }
+
+  private static class ByteArrayWriter extends ParquetValueWriters.PrimitiveWriter<byte[]> {
+    private ByteArrayWriter(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public void write(int repetitionLevel, byte[] bytes) {
+      column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes));
+    }
+  }
+
+  private static class ArrayDataWriter<E> extends ParquetValueWriters.RepeatedWriter<ArrayData, E> {
+    private final LogicalType elementType;
+
+    private ArrayDataWriter(int definitionLevel, int repetitionLevel,
+                            ParquetValueWriter<E> writer, LogicalType elementType) {
+      super(definitionLevel, repetitionLevel, writer);
+      this.elementType = elementType;
+    }
+
+    @Override
+    protected Iterator<E> elements(ArrayData list) {
+      return new ElementIterator<>(list);
+    }
+
+    private class ElementIterator<E> implements Iterator<E> {
+      private final int size;
+      private final ArrayData list;
+      private final ArrayData.ElementGetter getter;
+      private int index;
+
+      private ElementIterator(ArrayData list) {
+        this.list = list;
+        size = list.size();
+        getter = ArrayData.createElementGetter(elementType);
+        index = 0;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return index != size;
+      }
+
+      @Override
+      @SuppressWarnings("unchecked")
+      public E next() {
+        if (index >= size) {
+          throw new NoSuchElementException();
+        }
+
+        E element = (E) getter.getElementOrNull(list, index);
+        index += 1;
+
+        return element;
+      }
+    }
+  }
+
+  private static class MapDataWriter<K, V> extends ParquetValueWriters.RepeatedKeyValueWriter<MapData, K, V> {
+    private final LogicalType keyType;
+    private final LogicalType valueType;
+
+    private MapDataWriter(int definitionLevel, int repetitionLevel,
+                          ParquetValueWriter<K> keyWriter, ParquetValueWriter<V> valueWriter,
+                          LogicalType keyType, LogicalType valueType) {
+      super(definitionLevel, repetitionLevel, keyWriter, valueWriter);
+      this.keyType = keyType;
+      this.valueType = valueType;
+    }
+
+    @Override
+    protected Iterator<Map.Entry<K, V>> pairs(MapData map) {
+      return new EntryIterator<>(map);
+    }
+
+    private class EntryIterator<K, V> implements Iterator<Map.Entry<K, V>> {
+      private final int size;
+      private final ArrayData keys;
+      private final ArrayData values;
+      private final ParquetValueReaders.ReusableEntry<K, V> entry;
+      private final ArrayData.ElementGetter keyGetter;
+      private final ArrayData.ElementGetter valueGetter;
+      private int index;
+
+      private EntryIterator(MapData map) {
+        size = map.size();
+        keys = map.keyArray();
+        values = map.valueArray();
+        entry = new ParquetValueReaders.ReusableEntry<>();
+        keyGetter = ArrayData.createElementGetter(keyType);
+        valueGetter = ArrayData.createElementGetter(valueType);
+        index = 0;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return index != size;
+      }
+
+      @Override
+      @SuppressWarnings("unchecked")
+      public Map.Entry<K, V> next() {
+        if (index >= size) {
+          throw new NoSuchElementException();
+        }
+
+        entry.set((K) keyGetter.getElementOrNull(keys, index), (V) valueGetter.getElementOrNull(values, index));
+        index += 1;
+
+        return entry;
+      }
+    }
+  }
+
+  private static class RowDataWriter extends ParquetValueWriters.StructWriter<RowData> {
+    private final RowData.FieldGetter[] fieldGetter;
+
+    RowDataWriter(List<ParquetValueWriter<?>> writers, List<LogicalType> types) {
+      super(writers);
+      fieldGetter = new RowData.FieldGetter[types.size()];
+      for (int i = 0; i < types.size(); i += 1) {
+        fieldGetter[i] = RowData.createFieldGetter(types.get(i), i);
+      }
+    }
+
+    @Override
+    protected Object get(RowData struct, int index) {
+      return fieldGetter[index].getFieldOrNull(struct);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java
new file mode 100644
index 0000000..0909e1b
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java
@@ -0,0 +1,164 @@
+/*
+ * 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.data;
+
+import java.util.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+
+abstract class FlinkSchemaVisitor<T> {
+
+  static <T> T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor<T> visitor) {
+    return visit(flinkType, schema.asStruct(), visitor);
+  }
+
+  private static <T> T visit(LogicalType flinkType, Type iType, FlinkSchemaVisitor<T> visitor) {
+    switch (iType.typeId()) {
+      case STRUCT:
+        return visitRecord(flinkType, iType.asStructType(), visitor);
+
+      case MAP:
+        MapType mapType = (MapType) flinkType;
+        Types.MapType iMapType = iType.asMapType();
+        T key;
+        T value;
+
+        Types.NestedField keyField = iMapType.field(iMapType.keyId());
+        visitor.beforeMapKey(keyField);
+        try {
+          key = visit(mapType.getKeyType(), iMapType.keyType(), visitor);
+        } finally {
+          visitor.afterMapKey(keyField);
+        }
+
+        Types.NestedField valueField = iMapType.field(iMapType.valueId());
+        visitor.beforeMapValue(valueField);
+        try {
+          value = visit(mapType.getValueType(), iMapType.valueType(), visitor);
+        } finally {
+          visitor.afterMapValue(valueField);
+        }
+
+        return visitor.map(iMapType, key, value, mapType.getKeyType(), mapType.getValueType());
+
+      case LIST:
+        ArrayType listType = (ArrayType) flinkType;
+        Types.ListType iListType = iType.asListType();
+        T element;
+
+        Types.NestedField elementField = iListType.field(iListType.elementId());
+        visitor.beforeListElement(elementField);
+        try {
+          element = visit(listType.getElementType(), iListType.elementType(), visitor);
+        } finally {
+          visitor.afterListElement(elementField);
+        }
+
+        return visitor.list(iListType, element, listType.getElementType());
+
+      default:
+        return visitor.primitive(iType.asPrimitiveType(), flinkType);
+    }
+  }
+
+  private static <T> T visitRecord(LogicalType flinkType, Types.StructType struct,
+                                   FlinkSchemaVisitor<T> visitor) {
+    Preconditions.checkArgument(flinkType instanceof RowType, "%s is not a RowType.", flinkType);
+    RowType rowType = (RowType) flinkType;
+
+    int fieldSize = struct.fields().size();
+    List<T> results = Lists.newArrayListWithExpectedSize(fieldSize);
+    List<LogicalType> fieldTypes = Lists.newArrayListWithExpectedSize(fieldSize);
+    List<Types.NestedField> nestedFields = struct.fields();
+
+    for (int i = 0; i < fieldSize; i++) {
+      Types.NestedField iField = nestedFields.get(i);
+      int fieldIndex = rowType.getFieldIndex(iField.name());
+      Preconditions.checkArgument(fieldIndex >= 0,
+          "NestedField: %s is not found in flink RowType: %s", iField, rowType);
+
+      LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex);
+
+      fieldTypes.add(fieldFlinkType);
+
+      visitor.beforeField(iField);
+      try {
+        results.add(visit(fieldFlinkType, iField.type(), visitor));
+      } finally {
+        visitor.afterField(iField);
+      }
+    }
+
+    return visitor.record(struct, results, fieldTypes);
+  }
+
+  public T record(Types.StructType iStruct, List<T> results, List<LogicalType> fieldTypes) {
+    return null;
+  }
+
+  public T list(Types.ListType iList, T element, LogicalType elementType) {
+    return null;
+  }
+
+  public T map(Types.MapType iMap, T key, T value, LogicalType keyType, LogicalType valueType) {
+    return null;
+  }
+
+  public T primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) {
+    return null;
+  }
+
+  public void beforeField(Types.NestedField field) {
+  }
+
+  public void afterField(Types.NestedField field) {
+  }
+
+  public void beforeListElement(Types.NestedField elementField) {
+    beforeField(elementField);
+  }
+
+  public void afterListElement(Types.NestedField elementField) {
+    afterField(elementField);
+  }
+
+  public void beforeMapKey(Types.NestedField keyField) {
+    beforeField(keyField);
+  }
+
+  public void afterMapKey(Types.NestedField keyField) {
+    afterField(keyField);
+  }
+
+  public void beforeMapValue(Types.NestedField valueField) {
+    beforeField(valueField);
+  }
+
+  public void afterMapValue(Types.NestedField valueField) {
+    afterField(valueField);
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java
new file mode 100644
index 0000000..1b7a98f
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java
@@ -0,0 +1,313 @@
+/*
+ * 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.data;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.avro.ValueReader;
+import org.apache.iceberg.avro.ValueReaders;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+
+public class FlinkValueReaders {
+
+  private FlinkValueReaders() {
+  }
+
+  static ValueReader<StringData> strings() {
+    return StringReader.INSTANCE;
+  }
+
+  static ValueReader<StringData> enums(List<String> symbols) {
+    return new EnumReader(symbols);
+  }
+
+  static ValueReader<byte[]> uuids() {
+    return ValueReaders.fixed(16);
+  }
+
+  static ValueReader<Integer> timeMicros() {
+    return TimeMicrosReader.INSTANCE;
+  }
+
+  static ValueReader<TimestampData> timestampMills() {
+    return TimestampMillsReader.INSTANCE;
+  }
+
+  static ValueReader<TimestampData> timestampMicros() {
+    return TimestampMicrosReader.INSTANCE;
+  }
+
+  static ValueReader<DecimalData> decimal(ValueReader<byte[]> unscaledReader, int precision, int scale) {
+    return new DecimalReader(unscaledReader, precision, scale);
+  }
+
+  static ValueReader<ArrayData> array(ValueReader<?> elementReader) {
+    return new ArrayReader(elementReader);
+  }
+
+  static ValueReader<MapData> arrayMap(ValueReader<?> keyReader,
+                                       ValueReader<?> valueReader) {
+    return new ArrayMapReader(keyReader, valueReader);
+  }
+
+  static ValueReader<MapData> map(ValueReader<?> keyReader, ValueReader<?> valueReader) {
+    return new MapReader(keyReader, valueReader);
+  }
+
+  static ValueReader<RowData> struct(List<ValueReader<?>> readers, Types.StructType struct,
+                                     Map<Integer, ?> idToConstant) {
+    return new StructReader(readers, struct, idToConstant);
+  }
+
+  private static class StringReader implements ValueReader<StringData> {
+    private static final StringReader INSTANCE = new StringReader();
+
+    private StringReader() {
+    }
+
+    @Override
+    public StringData read(Decoder decoder, Object reuse) throws IOException {
+      // use the decoder's readString(Utf8) method because it may be a resolving decoder
+      Utf8 utf8 = null;
+      if (reuse instanceof StringData) {
+        utf8 = new Utf8(((StringData) reuse).toBytes());
+      }
+
+      Utf8 string = decoder.readString(utf8);
+      return StringData.fromBytes(string.getBytes(), 0, string.getByteLength());
+    }
+  }
+
+  private static class EnumReader implements ValueReader<StringData> {
+    private final StringData[] symbols;
+
+    private EnumReader(List<String> symbols) {
+      this.symbols = new StringData[symbols.size()];
+      for (int i = 0; i < this.symbols.length; i += 1) {
+        this.symbols[i] = StringData.fromBytes(symbols.get(i).getBytes(StandardCharsets.UTF_8));
+      }
+    }
+
+    @Override
+    public StringData read(Decoder decoder, Object ignore) throws IOException {
+      int index = decoder.readEnum();
+      return symbols[index];
+    }
+  }
+
+  private static class DecimalReader implements ValueReader<DecimalData> {
+    private final ValueReader<byte[]> bytesReader;
+    private final int precision;
+    private final int scale;
+
+    private DecimalReader(ValueReader<byte[]> bytesReader, int precision, int scale) {
+      this.bytesReader = bytesReader;
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public DecimalData read(Decoder decoder, Object reuse) throws IOException {
+      byte[] bytes = bytesReader.read(decoder, null);
+      return DecimalData.fromBigDecimal(new BigDecimal(new BigInteger(bytes), scale), precision, scale);
+    }
+  }
+
+  private static class TimeMicrosReader implements ValueReader<Integer> {
+    private static final TimeMicrosReader INSTANCE = new TimeMicrosReader();
+
+    @Override
+    public Integer read(Decoder decoder, Object reuse) throws IOException {
+      long micros = decoder.readLong();
+      // Flink only support time mills, just erase micros.
+      return (int) (micros / 1000);
+    }
+  }
+
+  private static class TimestampMillsReader implements ValueReader<TimestampData> {
+    private static final TimestampMillsReader INSTANCE = new TimestampMillsReader();
+
+    @Override
+    public TimestampData read(Decoder decoder, Object reuse) throws IOException {
+      return TimestampData.fromEpochMillis(decoder.readLong());
+    }
+  }
+
+  private static class TimestampMicrosReader implements ValueReader<TimestampData> {
+    private static final TimestampMicrosReader INSTANCE = new TimestampMicrosReader();
+
+    @Override
+    public TimestampData read(Decoder decoder, Object reuse) throws IOException {
+      long micros = decoder.readLong();
+      long mills = micros / 1000;
+      int nanos = ((int) (micros % 1000)) * 1000;
+      if (nanos < 0) {
+        nanos += 1_000_000;
+        mills -= 1;
+      }
+      return TimestampData.fromEpochMillis(mills, nanos);
+    }
+  }
+
+  private static class ArrayReader implements ValueReader<ArrayData> {
+    private final ValueReader<?> elementReader;
+    private final List<Object> reusedList = Lists.newArrayList();
+
+    private ArrayReader(ValueReader<?> elementReader) {
+      this.elementReader = elementReader;
+    }
+
+    @Override
+    public GenericArrayData read(Decoder decoder, Object reuse) throws IOException {
+      reusedList.clear();
+      long chunkLength = decoder.readArrayStart();
+
+      while (chunkLength > 0) {
+        for (int i = 0; i < chunkLength; i += 1) {
+          reusedList.add(elementReader.read(decoder, null));
+        }
+
+        chunkLength = decoder.arrayNext();
+      }
+
+      // this will convert the list to an array so it is okay to reuse the list
+      return new GenericArrayData(reusedList.toArray());
+    }
+  }
+
+  private static MapData kvArrayToMap(List<Object> keyList, List<Object> valueList) {
+    Map<Object, Object> map = Maps.newHashMap();
+    Object[] keys = keyList.toArray();
+    Object[] values = valueList.toArray();
+    for (int i = 0; i < keys.length; i++) {
+      map.put(keys[i], values[i]);
+    }
+
+    return new GenericMapData(map);
+  }
+
+  private static class ArrayMapReader implements ValueReader<MapData> {
+    private final ValueReader<?> keyReader;
+    private final ValueReader<?> valueReader;
+
+    private final List<Object> reusedKeyList = Lists.newArrayList();
+    private final List<Object> reusedValueList = Lists.newArrayList();
+
+    private ArrayMapReader(ValueReader<?> keyReader, ValueReader<?> valueReader) {
+      this.keyReader = keyReader;
+      this.valueReader = valueReader;
+    }
+
+    @Override
+    public MapData read(Decoder decoder, Object reuse) throws IOException {
+      reusedKeyList.clear();
+      reusedValueList.clear();
+
+      long chunkLength = decoder.readArrayStart();
+
+      while (chunkLength > 0) {
+        for (int i = 0; i < chunkLength; i += 1) {
+          reusedKeyList.add(keyReader.read(decoder, null));
+          reusedValueList.add(valueReader.read(decoder, null));
+        }
+
+        chunkLength = decoder.arrayNext();
+      }
+
+      return kvArrayToMap(reusedKeyList, reusedValueList);
+    }
+  }
+
+  private static class MapReader implements ValueReader<MapData> {
+    private final ValueReader<?> keyReader;
+    private final ValueReader<?> valueReader;
+
+    private final List<Object> reusedKeyList = Lists.newArrayList();
+    private final List<Object> reusedValueList = Lists.newArrayList();
+
+    private MapReader(ValueReader<?> keyReader, ValueReader<?> valueReader) {
+      this.keyReader = keyReader;
+      this.valueReader = valueReader;
+    }
+
+    @Override
+    public MapData read(Decoder decoder, Object reuse) throws IOException {
+      reusedKeyList.clear();
+      reusedValueList.clear();
+
+      long chunkLength = decoder.readMapStart();
+
+      while (chunkLength > 0) {
+        for (int i = 0; i < chunkLength; i += 1) {
+          reusedKeyList.add(keyReader.read(decoder, null));
+          reusedValueList.add(valueReader.read(decoder, null));
+        }
+
+        chunkLength = decoder.mapNext();
+      }
+
+      return kvArrayToMap(reusedKeyList, reusedValueList);
+    }
+  }
+
+  private static class StructReader extends ValueReaders.StructReader<RowData> {
+    private final int numFields;
+
+    private StructReader(List<ValueReader<?>> readers, Types.StructType struct, Map<Integer, ?> idToConstant) {
+      super(readers, struct, idToConstant);
+      this.numFields = readers.size();
+    }
+
+    @Override
+    protected RowData reuseOrCreate(Object reuse) {
+      if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) {
+        return (GenericRowData) reuse;
+      }
+      return new GenericRowData(numFields);
+    }
+
+    @Override
+    protected Object get(RowData struct, int pos) {
+      return null;
+    }
+
+    @Override
+    protected void set(RowData struct, int pos, Object value) {
+      ((GenericRowData) struct).setField(pos, value);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java
new file mode 100644
index 0000000..d17978f
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java
@@ -0,0 +1,241 @@
+/*
+ * 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.data;
+
+import java.io.IOException;
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.iceberg.avro.ValueWriter;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.DecimalUtil;
+
+public class FlinkValueWriters {
+
+  private FlinkValueWriters() {
+  }
+
+  static ValueWriter<StringData> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  static ValueWriter<Integer> timeMicros() {
+    return TimeMicrosWriter.INSTANCE;
+  }
+
+  static ValueWriter<TimestampData> timestampMicros() {
+    return TimestampMicrosWriter.INSTANCE;
+  }
+
+  static ValueWriter<DecimalData> decimal(int precision, int scale) {
+    return new DecimalWriter(precision, scale);
+  }
+
+  static <T> ValueWriter<ArrayData> array(ValueWriter<T> elementWriter, LogicalType elementType) {
+    return new ArrayWriter<>(elementWriter, elementType);
+  }
+
+  static <K, V> ValueWriter<MapData> arrayMap(ValueWriter<K> keyWriter, LogicalType keyType,
+                                              ValueWriter<V> valueWriter, LogicalType valueType) {
+    return new ArrayMapWriter<>(keyWriter, keyType, valueWriter, valueType);
+  }
+
+  static <K, V> ValueWriter<MapData> map(ValueWriter<K> keyWriter, LogicalType keyType,
+                                         ValueWriter<V> valueWriter, LogicalType valueType) {
+    return new MapWriter<>(keyWriter, keyType, valueWriter, valueType);
+  }
+
+  static ValueWriter<RowData> row(List<ValueWriter<?>> writers, List<LogicalType> types) {
+    return new RowWriter(writers, types);
+  }
+
+  private static class StringWriter implements ValueWriter<StringData> {
+    private static final StringWriter INSTANCE = new StringWriter();
+
+    private StringWriter() {
+    }
+
+    @Override
+    public void write(StringData s, Encoder encoder) throws IOException {
+      // toBytes is cheaper than Avro calling toString, which incurs encoding costs
+      encoder.writeString(new Utf8(s.toBytes()));
+    }
+  }
+
+  private static class DecimalWriter implements ValueWriter<DecimalData> {
+    private final int precision;
+    private final int scale;
+    private final ThreadLocal<byte[]> bytes;
+
+    private DecimalWriter(int precision, int scale) {
+      this.precision = precision;
+      this.scale = scale;
+      this.bytes = ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]);
+    }
+
+    @Override
+    public void write(DecimalData d, Encoder encoder) throws IOException {
+      encoder.writeFixed(DecimalUtil.toReusedFixLengthBytes(precision, scale, d.toBigDecimal(), bytes.get()));
+    }
+  }
+
+  private static class TimeMicrosWriter implements ValueWriter<Integer> {
+    private static final TimeMicrosWriter INSTANCE = new TimeMicrosWriter();
+
+    @Override
+    public void write(Integer timeMills, Encoder encoder) throws IOException {
+      encoder.writeLong(timeMills * 1000);
+    }
+  }
+
+  private static class TimestampMicrosWriter implements ValueWriter<TimestampData> {
+    private static final TimestampMicrosWriter INSTANCE = new TimestampMicrosWriter();
+
+    @Override
+    public void write(TimestampData timestampData, Encoder encoder) throws IOException {
+      long micros = timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000;
+      encoder.writeLong(micros);
+    }
+  }
+
+  private static class ArrayWriter<T> implements ValueWriter<ArrayData> {
+    private final ValueWriter<T> elementWriter;
+    private final ArrayData.ElementGetter elementGetter;
+
+    private ArrayWriter(ValueWriter<T> elementWriter, LogicalType elementType) {
+      this.elementWriter = elementWriter;
+      this.elementGetter = ArrayData.createElementGetter(elementType);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void write(ArrayData array, Encoder encoder) throws IOException {
+      encoder.writeArrayStart();
+      int numElements = array.size();
+      encoder.setItemCount(numElements);
+      for (int i = 0; i < numElements; i += 1) {
+        encoder.startItem();
+        elementWriter.write((T) elementGetter.getElementOrNull(array, i), encoder);
+      }
+      encoder.writeArrayEnd();
+    }
+  }
+
+  private static class ArrayMapWriter<K, V> implements ValueWriter<MapData> {
+    private final ValueWriter<K> keyWriter;
+    private final ValueWriter<V> valueWriter;
+    private final ArrayData.ElementGetter keyGetter;
+    private final ArrayData.ElementGetter valueGetter;
+
+    private ArrayMapWriter(ValueWriter<K> keyWriter, LogicalType keyType,
+                           ValueWriter<V> valueWriter, LogicalType valueType) {
+      this.keyWriter = keyWriter;
+      this.keyGetter = ArrayData.createElementGetter(keyType);
+      this.valueWriter = valueWriter;
+      this.valueGetter = ArrayData.createElementGetter(valueType);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void write(MapData map, Encoder encoder) throws IOException {
+      encoder.writeArrayStart();
+      int numElements = map.size();
+      encoder.setItemCount(numElements);
+      ArrayData keyArray = map.keyArray();
+      ArrayData valueArray = map.valueArray();
+      for (int i = 0; i < numElements; i += 1) {
+        encoder.startItem();
+        keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder);
+        valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder);
+      }
+      encoder.writeArrayEnd();
+    }
+  }
+
+  private static class MapWriter<K, V> implements ValueWriter<MapData> {
+    private final ValueWriter<K> keyWriter;
+    private final ValueWriter<V> valueWriter;
+    private final ArrayData.ElementGetter keyGetter;
+    private final ArrayData.ElementGetter valueGetter;
+
+    private MapWriter(ValueWriter<K> keyWriter, LogicalType keyType,
+                      ValueWriter<V> valueWriter, LogicalType valueType) {
+      this.keyWriter = keyWriter;
+      this.keyGetter = ArrayData.createElementGetter(keyType);
+      this.valueWriter = valueWriter;
+      this.valueGetter = ArrayData.createElementGetter(valueType);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void write(MapData map, Encoder encoder) throws IOException {
+      encoder.writeMapStart();
+      int numElements = map.size();
+      encoder.setItemCount(numElements);
+      ArrayData keyArray = map.keyArray();
+      ArrayData valueArray = map.valueArray();
+      for (int i = 0; i < numElements; i += 1) {
+        encoder.startItem();
+        keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder);
+        valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder);
+      }
+      encoder.writeMapEnd();
+    }
+  }
+
+  static class RowWriter implements ValueWriter<RowData> {
+    private final ValueWriter<?>[] writers;
+    private final RowData.FieldGetter[] getters;
+
+    private RowWriter(List<ValueWriter<?>> writers, List<LogicalType> types) {
+      this.writers = (ValueWriter<?>[]) Array.newInstance(ValueWriter.class, writers.size());
+      this.getters = new RowData.FieldGetter[writers.size()];
+      for (int i = 0; i < writers.size(); i += 1) {
+        this.writers[i] = writers.get(i);
+        this.getters[i] = RowData.createFieldGetter(types.get(i), i);
+      }
+    }
+
+    @Override
+    public void write(RowData row, Encoder encoder) throws IOException {
+      for (int i = 0; i < writers.length; i += 1) {
+        if (row.isNullAt(i)) {
+          writers[i].write(null, encoder);
+        } else {
+          write(row, i, writers[i], encoder);
+        }
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> void write(RowData row, int pos, ValueWriter<T> writer, Encoder encoder)
+        throws IOException {
+      writer.write((T) getters[pos].getFieldOrNull(row), encoder);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java
new file mode 100644
index 0000000..541986f
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java
@@ -0,0 +1,199 @@
+/*
+ * 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.data;
+
+import java.util.Deque;
+import java.util.List;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.RowType.RowField;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+public class ParquetWithFlinkSchemaVisitor<T> {
+  private final Deque<String> fieldNames = Lists.newLinkedList();
+
+  public static <T> T visit(LogicalType sType, Type type, ParquetWithFlinkSchemaVisitor<T> visitor) {
+    Preconditions.checkArgument(sType != null, "Invalid DataType: null");
+    if (type instanceof MessageType) {
+      Preconditions.checkArgument(sType instanceof RowType, "Invalid struct: %s is not a struct", sType);
+      RowType struct = (RowType) sType;
+      return visitor.message(struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor));
+    } else if (type.isPrimitive()) {
+      return visitor.primitive(sType, type.asPrimitiveType());
+    } else {
+      // if not a primitive, the typeId must be a group
+      GroupType group = type.asGroupType();
+      OriginalType annotation = group.getOriginalType();
+      if (annotation != null) {
+        switch (annotation) {
+          case LIST:
+            Preconditions.checkArgument(!group.isRepetition(Type.Repetition.REPEATED),
+                "Invalid list: top-level group is repeated: %s", group);
+            Preconditions.checkArgument(group.getFieldCount() == 1,
+                "Invalid list: does not contain single repeated field: %s", group);
+
+            GroupType repeatedElement = group.getFields().get(0).asGroupType();
+            Preconditions.checkArgument(repeatedElement.isRepetition(Type.Repetition.REPEATED),
+                "Invalid list: inner group is not repeated");
+            Preconditions.checkArgument(repeatedElement.getFieldCount() <= 1,
+                "Invalid list: repeated group is not a single field: %s", group);
+
+            Preconditions.checkArgument(sType instanceof ArrayType, "Invalid list: %s is not an array", sType);
+            ArrayType array = (ArrayType) sType;
+            RowType.RowField element = new RowField(
+                "element", array.getElementType(), "element of " + array.asSummaryString());
+
+            visitor.fieldNames.push(repeatedElement.getName());
+            try {
+              T elementResult = null;
+              if (repeatedElement.getFieldCount() > 0) {
+                elementResult = visitField(element, repeatedElement.getType(0), visitor);
+              }
+
+              return visitor.list(array, group, elementResult);
+
+            } finally {
+              visitor.fieldNames.pop();
+            }
+
+          case MAP:
+            Preconditions.checkArgument(!group.isRepetition(Type.Repetition.REPEATED),
+                "Invalid map: top-level group is repeated: %s", group);
+            Preconditions.checkArgument(group.getFieldCount() == 1,
+                "Invalid map: does not contain single repeated field: %s", group);
+
+            GroupType repeatedKeyValue = group.getType(0).asGroupType();
+            Preconditions.checkArgument(repeatedKeyValue.isRepetition(Type.Repetition.REPEATED),
+                "Invalid map: inner group is not repeated");
+            Preconditions.checkArgument(repeatedKeyValue.getFieldCount() <= 2,
+                "Invalid map: repeated group does not have 2 fields");
+
+            Preconditions.checkArgument(sType instanceof MapType, "Invalid map: %s is not a map", sType);
+            MapType map = (MapType) sType;
+            RowField keyField = new RowField("key", map.getKeyType(), "key of " + map.asSummaryString());
+            RowField valueField = new RowField(
+                "value", map.getValueType(), "value of " + map.asSummaryString());
+
+            visitor.fieldNames.push(repeatedKeyValue.getName());
+            try {
+              T keyResult = null;
+              T valueResult = null;
+              switch (repeatedKeyValue.getFieldCount()) {
+                case 2:
+                  // if there are 2 fields, both key and value are projected
+                  keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor);
+                  valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor);
+                  break;
+                case 1:
+                  // if there is just one, use the name to determine what it is
+                  Type keyOrValue = repeatedKeyValue.getType(0);
+                  if (keyOrValue.getName().equalsIgnoreCase("key")) {
+                    keyResult = visitField(keyField, keyOrValue, visitor);
+                    // value result remains null
+                  } else {
+                    valueResult = visitField(valueField, keyOrValue, visitor);
+                    // key result remains null
+                  }
+                  break;
+                default:
+                  // both results will remain null
+              }
+
+              return visitor.map(map, group, keyResult, valueResult);
+
+            } finally {
+              visitor.fieldNames.pop();
+            }
+
+          default:
+        }
+      }
+      Preconditions.checkArgument(sType instanceof RowType, "Invalid struct: %s is not a struct", sType);
+      RowType struct = (RowType) sType;
+      return visitor.struct(struct, group, visitFields(struct, group, visitor));
+    }
+  }
+
+  private static <T> T visitField(RowType.RowField sField, Type field, ParquetWithFlinkSchemaVisitor<T> visitor) {
+    visitor.fieldNames.push(field.getName());
+    try {
+      return visit(sField.getType(), field, visitor);
+    } finally {
+      visitor.fieldNames.pop();
+    }
+  }
+
+  private static <T> List<T> visitFields(RowType struct, GroupType group,
+                                         ParquetWithFlinkSchemaVisitor<T> visitor) {
+    List<RowType.RowField> sFields = struct.getFields();
+    Preconditions.checkArgument(sFields.size() == group.getFieldCount(),
+        "Structs do not match: %s and %s", struct, group);
+    List<T> results = Lists.newArrayListWithExpectedSize(group.getFieldCount());
+    for (int i = 0; i < sFields.size(); i += 1) {
+      Type field = group.getFields().get(i);
+      RowType.RowField sField = sFields.get(i);
+      Preconditions.checkArgument(field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.getName())),
+          "Structs do not match: field %s != %s", field.getName(), sField.getName());
+      results.add(visitField(sField, field, visitor));
+    }
+
+    return results;
+  }
+
+  public T message(RowType sStruct, MessageType message, List<T> fields) {
+    return null;
+  }
+
+  public T struct(RowType sStruct, GroupType struct, List<T> fields) {
+    return null;
+  }
+
+  public T list(ArrayType sArray, GroupType array, T element) {
+    return null;
+  }
+
+  public T map(MapType sMap, GroupType map, T key, T value) {
+    return null;
+  }
+
+  public T primitive(LogicalType sPrimitive, PrimitiveType primitive) {
+    return null;
+  }
+
+  protected String[] currentPath() {
+    return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]);
+  }
+
+  protected String[] path(String name) {
+    List<String> list = Lists.newArrayList(fieldNames.descendingIterator());
+    list.add(name);
+    return list.toArray(new String[0]);
+  }
+
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java
new file mode 100644
index 0000000..6334a00
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java
@@ -0,0 +1,239 @@
+/*
+ * 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.data;
+
+import java.util.Map;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RawValueData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+
+public class RowDataProjection implements RowData {
+  /**
+   * Creates a projecting wrapper for {@link RowData} rows.
+   * <p>
+   * This projection will not project the nested children types of repeated types like lists and maps.
+   *
+   * @param schema schema of rows wrapped by this projection
+   * @param projectedSchema result schema of the projected rows
+   * @return a wrapper to project rows
+   */
+  public static RowDataProjection create(Schema schema, Schema projectedSchema) {
+    return RowDataProjection.create(FlinkSchemaUtil.convert(schema), schema.asStruct(), projectedSchema.asStruct());
+  }
+
+  /**
+   * Creates a projecting wrapper for {@link RowData} rows.
+   * <p>
+   * This projection will not project the nested children types of repeated types like lists and maps.
+   *
+   * @param rowType flink row type of rows wrapped by this projection
+   * @param schema schema of rows wrapped by this projection
+   * @param projectedSchema result schema of the projected rows
+   * @return a wrapper to project rows
+   */
+  public static RowDataProjection create(RowType rowType, Types.StructType schema, Types.StructType projectedSchema) {
+    return new RowDataProjection(rowType, schema, projectedSchema);
+  }
+
+  private final RowData.FieldGetter[] getters;
+  private RowData rowData;
+
+  private RowDataProjection(RowType rowType, Types.StructType rowStruct, Types.StructType projectType) {
+    Map<Integer, Integer> fieldIdToPosition = Maps.newHashMap();
+    for (int i = 0; i < rowStruct.fields().size(); i++) {
+      fieldIdToPosition.put(rowStruct.fields().get(i).fieldId(), i);
+    }
+
+    this.getters = new RowData.FieldGetter[projectType.fields().size()];
+    for (int i = 0; i < getters.length; i++) {
+      Types.NestedField projectField = projectType.fields().get(i);
+      Types.NestedField rowField = rowStruct.field(projectField.fieldId());
+
+      Preconditions.checkNotNull(rowField,
+          "Cannot locate the project field <%s> in the iceberg struct <%s>", projectField, rowStruct);
+
+      getters[i] = createFieldGetter(rowType, fieldIdToPosition.get(projectField.fieldId()), rowField, projectField);
+    }
+  }
+
+  private static RowData.FieldGetter createFieldGetter(RowType rowType,
+                                                       int position,
+                                                       Types.NestedField rowField,
+                                                       Types.NestedField projectField) {
+    Preconditions.checkArgument(rowField.type().typeId() == projectField.type().typeId(),
+        "Different iceberg type between row field <%s> and project field <%s>", rowField, projectField);
+
+    switch (projectField.type().typeId()) {
+      case STRUCT:
+        RowType nestedRowType = (RowType) rowType.getTypeAt(position);
+        return row -> {
+          RowData nestedRow = row.isNullAt(position) ? null : row.getRow(position, nestedRowType.getFieldCount());
+          return RowDataProjection
+              .create(nestedRowType, rowField.type().asStructType(), projectField.type().asStructType())
+              .wrap(nestedRow);
+        };
+
+      case MAP:
+        Types.MapType projectedMap = projectField.type().asMapType();
+        Types.MapType originalMap = rowField.type().asMapType();
+
+        boolean keyProjectable = !projectedMap.keyType().isNestedType() ||
+            projectedMap.keyType().equals(originalMap.keyType());
+        boolean valueProjectable = !projectedMap.valueType().isNestedType() ||
+            projectedMap.valueType().equals(originalMap.valueType());
+        Preconditions.checkArgument(keyProjectable && valueProjectable,
+            "Cannot project a partial map key or value with non-primitive type. Trying to project <%s> out of <%s>",
+            projectField, rowField);
+
+        return RowData.createFieldGetter(rowType.getTypeAt(position), position);
+
+      case LIST:
+        Types.ListType projectedList = projectField.type().asListType();
+        Types.ListType originalList = rowField.type().asListType();
+
+        boolean elementProjectable = !projectedList.elementType().isNestedType() ||
+            projectedList.elementType().equals(originalList.elementType());
+        Preconditions.checkArgument(elementProjectable,
+            "Cannot project a partial list element with non-primitive type. Trying to project <%s> out of <%s>",
+            projectField, rowField);
+
+        return RowData.createFieldGetter(rowType.getTypeAt(position), position);
+
+      default:
+        return RowData.createFieldGetter(rowType.getTypeAt(position), position);
+    }
+  }
+
+  public RowData wrap(RowData row) {
+    this.rowData = row;
+    return this;
+  }
+
+  private Object getValue(int pos) {
+    return getters[pos].getFieldOrNull(rowData);
+  }
+
+  @Override
+  public int getArity() {
+    return getters.length;
+  }
+
+  @Override
+  public RowKind getRowKind() {
+    return rowData.getRowKind();
+  }
+
+  @Override
+  public void setRowKind(RowKind kind) {
+    throw new UnsupportedOperationException("Cannot set row kind in the RowDataProjection");
+  }
+
+  @Override
+  public boolean isNullAt(int pos) {
+    return rowData == null || getValue(pos) == null;
+  }
+
+  @Override
+  public boolean getBoolean(int pos) {
+    return (boolean) getValue(pos);
+  }
+
+  @Override
+  public byte getByte(int pos) {
+    return (byte) getValue(pos);
+  }
+
+  @Override
+  public short getShort(int pos) {
+    return (short) getValue(pos);
+  }
+
+  @Override
+  public int getInt(int pos) {
+    return (int) getValue(pos);
+  }
+
+  @Override
+  public long getLong(int pos) {
+    return (long) getValue(pos);
+  }
+
+  @Override
+  public float getFloat(int pos) {
+    return (float) getValue(pos);
+  }
+
+  @Override
+  public double getDouble(int pos) {
+    return (double) getValue(pos);
+  }
+
+  @Override
+  public StringData getString(int pos) {
+    return (StringData) getValue(pos);
+  }
+
+  @Override
+  public DecimalData getDecimal(int pos, int precision, int scale) {
+    return (DecimalData) getValue(pos);
+  }
+
+  @Override
+  public TimestampData getTimestamp(int pos, int precision) {
+    return (TimestampData) getValue(pos);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T> RawValueData<T> getRawValue(int pos) {
+    return (RawValueData<T>) getValue(pos);
+  }
+
+  @Override
+  public byte[] getBinary(int pos) {
+    return (byte[]) getValue(pos);
+  }
+
+  @Override
+  public ArrayData getArray(int pos) {
+    return (ArrayData) getValue(pos);
+  }
+
+  @Override
+  public MapData getMap(int pos) {
+    return (MapData) getValue(pos);
+  }
+
+  @Override
+  public RowData getRow(int pos, int numFields) {
+    return (RowData) getValue(pos);
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java
new file mode 100644
index 0000000..931880f
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java
@@ -0,0 +1,103 @@
+/*
+ * 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.data;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.data.DecimalData;
+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.TimestampData;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+public class RowDataUtil {
+
+  private RowDataUtil() {
+
+  }
+
+  public static Object convertConstant(Type type, Object value) {
+    if (value == null) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case DECIMAL: // DecimalData
+        Types.DecimalType decimal = (Types.DecimalType) type;
+        return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale());
+      case STRING: // StringData
+        if (value instanceof Utf8) {
+          Utf8 utf8 = (Utf8) value;
+          return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength());
+        }
+        return StringData.fromString(value.toString());
+      case FIXED: // byte[]
+        if (value instanceof byte[]) {
+          return value;
+        } else if (value instanceof GenericData.Fixed) {
+          return ((GenericData.Fixed) value).bytes();
+        }
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case BINARY: // byte[]
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case TIME: // int mills instead of long
+        return (int) ((Long) value / 1000);
+      case TIMESTAMP: // TimestampData
+        return TimestampData.fromLocalDateTime(DateTimeUtil.timestampFromMicros((Long) value));
+      default:
+    }
+    return value;
+  }
+
+  /**
+   * Similar to the private {@link RowDataSerializer#copyRowData(RowData, RowData)} method.
+   * This skips the check the arity of rowType and from,
+   * because the from RowData may contains additional column for position deletes.
+   * Using {@link RowDataSerializer#copy(RowData, RowData)} will fail the arity check.
+   */
+  public static RowData clone(RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) {
+    GenericRowData ret;
+    if (reuse instanceof GenericRowData) {
+      ret = (GenericRowData) reuse;
+    } else {
+      ret = new GenericRowData(from.getArity());
+    }
+    ret.setRowKind(from.getRowKind());
+    for (int i = 0; i < rowType.getFieldCount(); i++) {
+      if (!from.isNullAt(i)) {
+        RowData.FieldGetter getter = RowData.createFieldGetter(rowType.getTypeAt(i), i);
+        ret.setField(i, fieldSerializers[i].copy(getter.getFieldOrNull(from)));
+      } else {
+        ret.setField(i, null);
+      }
+    }
+    return ret;
+  }
+
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java
new file mode 100644
index 0000000..8415129
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java
@@ -0,0 +1,107 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.RowDataWrapper;
+import org.apache.iceberg.io.BaseTaskWriter;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+
+abstract class BaseDeltaTaskWriter extends BaseTaskWriter<RowData> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+  private final RowDataWrapper wrapper;
+  private final boolean upsert;
+
+  BaseDeltaTaskWriter(PartitionSpec spec,
+                      FileFormat format,
+                      FileAppenderFactory<RowData> appenderFactory,
+                      OutputFileFactory fileFactory,
+                      FileIO io,
+                      long targetFileSize,
+                      Schema schema,
+                      RowType flinkSchema,
+                      List<Integer> equalityFieldIds,
+                      boolean upsert) {
+    super(spec, format, appenderFactory, fileFactory, io, targetFileSize);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+    this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct());
+    this.upsert = upsert;
+  }
+
+  abstract RowDataDeltaWriter route(RowData row);
+
+  RowDataWrapper wrapper() {
+    return wrapper;
+  }
+
+  @Override
+  public void write(RowData row) throws IOException {
+    RowDataDeltaWriter writer = route(row);
+
+    switch (row.getRowKind()) {
+      case INSERT:
+      case UPDATE_AFTER:
+        if (upsert) {
+          writer.delete(row);
+        }
+        writer.write(row);
+        break;
+
+      case UPDATE_BEFORE:
+        if (upsert) {
+          break;  // UPDATE_BEFORE is not necessary for UPDATE, we do nothing to prevent delete one row twice
+        }
+        writer.delete(row);
+        break;
+      case DELETE:
+        writer.delete(row);
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Unknown row kind: " + row.getRowKind());
+    }
+  }
+
+  protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter {
+    RowDataDeltaWriter(PartitionKey partition) {
+      super(partition, schema, deleteSchema);
+    }
+
+    @Override
+    protected StructLike asStructLike(RowData data) {
+      return wrapper.wrap(data);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java
new file mode 100644
index 0000000..866b785
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java
@@ -0,0 +1,71 @@
+/*
+ * 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;
+
+import java.util.List;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+class DeltaManifests {
+
+  private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0];
+
+  private final ManifestFile dataManifest;
+  private final ManifestFile deleteManifest;
+  private final CharSequence[] referencedDataFiles;
+
+  DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) {
+    this(dataManifest, deleteManifest, EMPTY_REF_DATA_FILES);
+  }
+
+  DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest, CharSequence[] referencedDataFiles) {
+    Preconditions.checkNotNull(referencedDataFiles, "Referenced data files shouldn't be null.");
+
+    this.dataManifest = dataManifest;
+    this.deleteManifest = deleteManifest;
+    this.referencedDataFiles = referencedDataFiles;
+  }
+
+  ManifestFile dataManifest() {
+    return dataManifest;
+  }
+
+  ManifestFile deleteManifest() {
+    return deleteManifest;
+  }
+
+  CharSequence[] referencedDataFiles() {
+    return referencedDataFiles;
+  }
+
+  List<ManifestFile> manifests() {
+    List<ManifestFile> manifests = Lists.newArrayListWithCapacity(2);
+    if (dataManifest != null) {
+      manifests.add(dataManifest);
+    }
+
+    if (deleteManifest != null) {
+      manifests.add(deleteManifest);
+    }
+
+    return manifests;
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java
new file mode 100644
index 0000000..859f979
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java
@@ -0,0 +1,122 @@
+/*
+ * 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;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+class DeltaManifestsSerializer implements SimpleVersionedSerializer<DeltaManifests> {
+  private static final int VERSION_1 = 1;
+  private static final int VERSION_2 = 2;
+  private static final byte[] EMPTY_BINARY = new byte[0];
+
+  static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer();
+
+  @Override
+  public int getVersion() {
+    return VERSION_2;
+  }
+
+  @Override
+  public byte[] serialize(DeltaManifests deltaManifests) throws IOException {
+    Preconditions.checkNotNull(deltaManifests, "DeltaManifests to be serialized should not be null");
+
+    ByteArrayOutputStream binaryOut = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(binaryOut);
+
+    byte[] dataManifestBinary = EMPTY_BINARY;
+    if (deltaManifests.dataManifest() != null) {
+      dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest());
+    }
+
+    out.writeInt(dataManifestBinary.length);
+    out.write(dataManifestBinary);
+
+    byte[] deleteManifestBinary = EMPTY_BINARY;
+    if (deltaManifests.deleteManifest() != null) {
+      deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest());
+    }
+
+    out.writeInt(deleteManifestBinary.length);
+    out.write(deleteManifestBinary);
+
+    CharSequence[] referencedDataFiles = deltaManifests.referencedDataFiles();
+    out.writeInt(referencedDataFiles.length);
+    for (int i = 0; i < referencedDataFiles.length; i++) {
+      out.writeUTF(referencedDataFiles[i].toString());
+    }
+
+    return binaryOut.toByteArray();
+  }
+
+  @Override
+  public DeltaManifests deserialize(int version, byte[] serialized) throws IOException {
+    if (version == VERSION_1) {
+      return deserializeV1(serialized);
+    } else if (version == VERSION_2) {
+      return deserializeV2(serialized);
+    } else {
+      throw new RuntimeException("Unknown serialize version: " + version);
+    }
+  }
+
+  private DeltaManifests deserializeV1(byte[] serialized) throws IOException {
+    return new DeltaManifests(ManifestFiles.decode(serialized), null);
+  }
+
+  private DeltaManifests deserializeV2(byte[] serialized) throws IOException {
+    ManifestFile dataManifest = null;
+    ManifestFile deleteManifest = null;
+
+    ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized);
+    DataInputStream in = new DataInputStream(binaryIn);
+
+    int dataManifestSize = in.readInt();
+    if (dataManifestSize > 0) {
+      byte[] dataManifestBinary = new byte[dataManifestSize];
+      Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize);
+
+      dataManifest = ManifestFiles.decode(dataManifestBinary);
+    }
+
+    int deleteManifestSize = in.readInt();
+    if (deleteManifestSize > 0) {
+      byte[] deleteManifestBinary = new byte[deleteManifestSize];
+      Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize);
+
+      deleteManifest = ManifestFiles.decode(deleteManifestBinary);
+    }
+
+    int referenceDataFileNum = in.readInt();
+    CharSequence[] referencedDataFiles = new CharSequence[referenceDataFileNum];
+    for (int i = 0; i < referenceDataFileNum; i++) {
+      referencedDataFiles[i] = in.readUTF();
+    }
+
+    return new DeltaManifests(dataManifest, deleteManifest, referencedDataFiles);
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java
new file mode 100644
index 0000000..ade5c28
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java
@@ -0,0 +1,251 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.data.FlinkAvroWriter;
+import org.apache.iceberg.flink.data.FlinkOrcWriter;
+import org.apache.iceberg.flink.data.FlinkParquetWriters;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FlinkAppenderFactory implements FileAppenderFactory<RowData>, Serializable {
+  private final Schema schema;
+  private final RowType flinkSchema;
+  private final Map<String, String> props;
+  private final PartitionSpec spec;
+  private final int[] equalityFieldIds;
+  private final Schema eqDeleteRowSchema;
+  private final Schema posDeleteRowSchema;
+
+  private RowType eqDeleteFlinkSchema = null;
+  private RowType posDeleteFlinkSchema = null;
+
+  public FlinkAppenderFactory(Schema schema, RowType flinkSchema, Map<String, String> props, PartitionSpec spec) {
+    this(schema, flinkSchema, props, spec, null, null, null);
+  }
+
+  public FlinkAppenderFactory(Schema schema, RowType flinkSchema, Map<String, String> props,
+                              PartitionSpec spec, int[] equalityFieldIds,
+                              Schema eqDeleteRowSchema, Schema posDeleteRowSchema) {
+    this.schema = schema;
+    this.flinkSchema = flinkSchema;
+    this.props = props;
+    this.spec = spec;
+    this.equalityFieldIds = equalityFieldIds;
+    this.eqDeleteRowSchema = eqDeleteRowSchema;
+    this.posDeleteRowSchema = posDeleteRowSchema;
+  }
+
+  private RowType lazyEqDeleteFlinkSchema() {
+    if (eqDeleteFlinkSchema == null) {
+      Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null");
+      this.eqDeleteFlinkSchema = FlinkSchemaUtil.convert(eqDeleteRowSchema);
+    }
+    return eqDeleteFlinkSchema;
+  }
+
+  private RowType lazyPosDeleteFlinkSchema() {
+    if (posDeleteFlinkSchema == null) {
+      Preconditions.checkNotNull(posDeleteRowSchema, "Pos-delete row schema shouldn't be null");
+      this.posDeleteFlinkSchema = FlinkSchemaUtil.convert(posDeleteRowSchema);
+    }
+    return this.posDeleteFlinkSchema;
+  }
+
+  @Override
+  public FileAppender<RowData> newAppender(OutputFile outputFile, FileFormat format) {
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(props);
+    try {
+      switch (format) {
+        case AVRO:
+          return Avro.write(outputFile)
+              .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema))
+              .setAll(props)
+              .schema(schema)
+              .metricsConfig(metricsConfig)
+              .overwrite()
+              .build();
+
+        case ORC:
+          return ORC.write(outputFile)
+              .createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema))
+              .setAll(props)
+              .metricsConfig(metricsConfig)
+              .schema(schema)
+              .overwrite()
+              .build();
+
+        case PARQUET:
+          return Parquet.write(outputFile)
+              .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType))
+              .setAll(props)
+              .metricsConfig(metricsConfig)
+              .schema(schema)
+              .overwrite()
+              .build();
+
+        default:
+          throw new UnsupportedOperationException("Cannot write unknown file format: " + format);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public DataWriter<RowData> newDataWriter(EncryptedOutputFile file, FileFormat format, StructLike partition) {
+    return new DataWriter<>(
+        newAppender(file.encryptingOutputFile(), format), format,
+        file.encryptingOutputFile().location(), spec, partition, file.keyMetadata());
+  }
+
+  @Override
+  public EqualityDeleteWriter<RowData> newEqDeleteWriter(EncryptedOutputFile outputFile, FileFormat format,
+                                                         StructLike partition) {
+    Preconditions.checkState(equalityFieldIds != null && equalityFieldIds.length > 0,
+        "Equality field ids shouldn't be null or empty when creating equality-delete writer");
+    Preconditions.checkNotNull(eqDeleteRowSchema,
+        "Equality delete row schema shouldn't be null when creating equality-delete writer");
+
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(props);
+    try {
+      switch (format) {
+        case AVRO:
+          return Avro.writeDeletes(outputFile.encryptingOutputFile())
+              .createWriterFunc(ignore -> new FlinkAvroWriter(lazyEqDeleteFlinkSchema()))
+              .withPartition(partition)
+              .overwrite()
+              .setAll(props)
+              .rowSchema(eqDeleteRowSchema)
+              .withSpec(spec)
+              .withKeyMetadata(outputFile.keyMetadata())
+              .equalityFieldIds(equalityFieldIds)
+              .buildEqualityWriter();
+
+        case ORC:
+          return ORC.writeDeletes(outputFile.encryptingOutputFile())
+              .createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema))
+              .withPartition(partition)
+              .overwrite()
+              .setAll(props)
+              .rowSchema(eqDeleteRowSchema)
+              .withSpec(spec)
+              .withKeyMetadata(outputFile.keyMetadata())
+              .equalityFieldIds(equalityFieldIds)
+              .buildEqualityWriter();
+
+        case PARQUET:
+          return Parquet.writeDeletes(outputFile.encryptingOutputFile())
+              .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(lazyEqDeleteFlinkSchema(), msgType))
+              .withPartition(partition)
+              .overwrite()
+              .setAll(props)
+              .metricsConfig(metricsConfig)
+              .rowSchema(eqDeleteRowSchema)
+              .withSpec(spec)
+              .withKeyMetadata(outputFile.keyMetadata())
+              .equalityFieldIds(equalityFieldIds)
+              .buildEqualityWriter();
+
+        default:
+          throw new UnsupportedOperationException(
+              "Cannot write equality-deletes for unsupported file format: " + format);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public PositionDeleteWriter<RowData> newPosDeleteWriter(EncryptedOutputFile outputFile, FileFormat format,
+                                                          StructLike partition) {
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(props);
+    try {
+      switch (format) {
+        case AVRO:
+          return Avro.writeDeletes(outputFile.encryptingOutputFile())
+              .createWriterFunc(ignore -> new FlinkAvroWriter(lazyPosDeleteFlinkSchema()))
+              .withPartition(partition)
+              .overwrite()
+              .setAll(props)
+              .rowSchema(posDeleteRowSchema)
+              .withSpec(spec)
+              .withKeyMetadata(outputFile.keyMetadata())
+              .buildPositionWriter();
+
+        case ORC:
+          RowType orcPosDeleteSchema = FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema));
+          return ORC.writeDeletes(outputFile.encryptingOutputFile())
+              .createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(orcPosDeleteSchema, iSchema))
+              .withPartition(partition)
+              .overwrite()
+              .setAll(props)
+              .metricsConfig(metricsConfig)
+              .rowSchema(posDeleteRowSchema)
+              .withSpec(spec)
+              .withKeyMetadata(outputFile.keyMetadata())
+              .transformPaths(path -> StringData.fromString(path.toString()))
+              .buildPositionWriter();
+
+        case PARQUET:
+          RowType flinkPosDeleteSchema = FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema));
+          return Parquet.writeDeletes(outputFile.encryptingOutputFile())
+              .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkPosDeleteSchema, msgType))
+              .withPartition(partition)
+              .overwrite()
+              .setAll(props)
+              .metricsConfig(metricsConfig)
+              .rowSchema(posDeleteRowSchema)
+              .withSpec(spec)
+              .withKeyMetadata(outputFile.keyMetadata())
+              .transformPaths(path -> StringData.fromString(path.toString()))
+              .buildPositionWriter();
+
+        default:
+          throw new UnsupportedOperationException("Cannot write pos-deletes for unsupported file format: " + format);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java
new file mode 100644
index 0000000..55a9539
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java
@@ -0,0 +1,259 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.BaseFileWriterFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.data.FlinkAvroWriter;
+import org.apache.iceberg.flink.data.FlinkOrcWriter;
+import org.apache.iceberg.flink.data.FlinkParquetWriters;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;
+
+class FlinkFileWriterFactory extends BaseFileWriterFactory<RowData> implements Serializable {
+  private RowType dataFlinkType;
+  private RowType equalityDeleteFlinkType;
+  private RowType positionDeleteFlinkType;
+
+  FlinkFileWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema, RowType dataFlinkType,
+                         SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                         int[] equalityFieldIds, Schema equalityDeleteRowSchema, RowType equalityDeleteFlinkType,
+                         SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                         RowType positionDeleteFlinkType) {
+
+    super(table, dataFileFormat, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds,
+        equalityDeleteRowSchema, equalityDeleteSortOrder, positionDeleteRowSchema);
+
+    this.dataFlinkType = dataFlinkType;
+    this.equalityDeleteFlinkType = equalityDeleteFlinkType;
+    this.positionDeleteFlinkType = positionDeleteFlinkType;
+  }
+
+  static Builder builderFor(Table table) {
+    return new Builder(table);
+  }
+
+  @Override
+  protected void configureDataWrite(Avro.DataWriteBuilder builder) {
+    builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType()));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType()));
+  }
+
+  @Override
+  protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {
+    int rowFieldIndex = positionDeleteFlinkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME);
+    if (rowFieldIndex >= 0) {
+      // FlinkAvroWriter accepts just the Flink type of the row ignoring the path and pos
+      RowType positionDeleteRowFlinkType = (RowType) positionDeleteFlinkType().getTypeAt(rowFieldIndex);
+      builder.createWriterFunc(ignored -> new FlinkAvroWriter(positionDeleteRowFlinkType));
+    }
+  }
+
+  @Override
+  protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType));
+  }
+
+  @Override
+  protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(positionDeleteFlinkType(), msgType));
+    builder.transformPaths(path -> StringData.fromString(path.toString()));
+  }
+
+  @Override
+  protected void configureDataWrite(ORC.DataWriteBuilder builder) {
+    builder.createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema));
+  }
+
+  @Override
+  protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) {
+    builder.createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema));
+  }
+
+  @Override
+  protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) {
+    builder.createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(positionDeleteFlinkType(), iSchema));
+    builder.transformPaths(path -> StringData.fromString(path.toString()));
+  }
+
+  private RowType dataFlinkType() {
+    if (dataFlinkType == null) {
+      Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
+      this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema());
+    }
+
+    return dataFlinkType;
+  }
+
+  private RowType equalityDeleteFlinkType() {
+    if (equalityDeleteFlinkType == null) {
+      Preconditions.checkNotNull(equalityDeleteRowSchema(), "Equality delete schema must not be null");
+      this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema());
+    }
+
+    return equalityDeleteFlinkType;
+  }
+
+  private RowType positionDeleteFlinkType() {
+    if (positionDeleteFlinkType == null) {
+      // wrap the optional row schema into the position delete schema that contains path and position
+      Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema());
+      this.positionDeleteFlinkType = FlinkSchemaUtil.convert(positionDeleteSchema);
+    }
+
+    return positionDeleteFlinkType;
+  }
+
+  static class Builder {
+    private final Table table;
+    private FileFormat dataFileFormat;
+    private Schema dataSchema;
+    private RowType dataFlinkType;
+    private SortOrder dataSortOrder;
+    private FileFormat deleteFileFormat;
+    private int[] equalityFieldIds;
+    private Schema equalityDeleteRowSchema;
+    private RowType equalityDeleteFlinkType;
+    private SortOrder equalityDeleteSortOrder;
+    private Schema positionDeleteRowSchema;
+    private RowType positionDeleteFlinkType;
+
+    Builder(Table table) {
+      this.table = table;
+
+      Map<String, String> properties = table.properties();
+
+      String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+      this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
+
+      String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
+      this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
+    }
+
+    Builder dataFileFormat(FileFormat newDataFileFormat) {
+      this.dataFileFormat = newDataFileFormat;
+      return this;
+    }
+
+    Builder dataSchema(Schema newDataSchema) {
+      this.dataSchema = newDataSchema;
+      return this;
+    }
+
+    /**
+     * Sets a Flink type for data.
+     * <p>
+     * If not set, the value is derived from the provided Iceberg schema.
+     */
+    Builder dataFlinkType(RowType newDataFlinkType) {
+      this.dataFlinkType = newDataFlinkType;
+      return this;
+    }
+
+    Builder dataSortOrder(SortOrder newDataSortOrder) {
+      this.dataSortOrder = newDataSortOrder;
+      return this;
+    }
+
+    Builder deleteFileFormat(FileFormat newDeleteFileFormat) {
+      this.deleteFileFormat = newDeleteFileFormat;
+      return this;
+    }
+
+    Builder equalityFieldIds(int[] newEqualityFieldIds) {
+      this.equalityFieldIds = newEqualityFieldIds;
+      return this;
+    }
+
+    Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) {
+      this.equalityDeleteRowSchema = newEqualityDeleteRowSchema;
+      return this;
+    }
+
+    /**
+     * Sets a Flink type for equality deletes.
+     * <p>
+     * If not set, the value is derived from the provided Iceberg schema.
+     */
+    Builder equalityDeleteFlinkType(RowType newEqualityDeleteFlinkType) {
+      this.equalityDeleteFlinkType = newEqualityDeleteFlinkType;
+      return this;
+    }
+
+    Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) {
+      this.equalityDeleteSortOrder = newEqualityDeleteSortOrder;
+      return this;
+    }
+
+    Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) {
+      this.positionDeleteRowSchema = newPositionDeleteRowSchema;
+      return this;
+    }
+
+    /**
+     * Sets a Flink type for position deletes.
+     * <p>
+     * If not set, the value is derived from the provided Iceberg schema.
+     */
+    Builder positionDeleteFlinkType(RowType newPositionDeleteFlinkType) {
+      this.positionDeleteFlinkType = newPositionDeleteFlinkType;
+      return this;
+    }
+
+    FlinkFileWriterFactory build() {
+      boolean noEqualityDeleteConf = equalityFieldIds == null && equalityDeleteRowSchema == null;
+      boolean fullEqualityDeleteConf = equalityFieldIds != null && equalityDeleteRowSchema != null;
+      Preconditions.checkArgument(noEqualityDeleteConf || fullEqualityDeleteConf,
+          "Equality field IDs and equality delete row schema must be set together");
+
+      return new FlinkFileWriterFactory(
+          table, dataFileFormat, dataSchema, dataFlinkType, dataSortOrder, deleteFileFormat,
+          equalityFieldIds, equalityDeleteRowSchema, equalityDeleteFlinkType, equalityDeleteSortOrder,
+          positionDeleteRowSchema, positionDeleteFlinkType);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java
new file mode 100644
index 0000000..b00018b
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.function.Supplier;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+class FlinkManifestUtil {
+  private static final int FORMAT_V2 = 2;
+  private static final Long DUMMY_SNAPSHOT_ID = 0L;
+
+  private FlinkManifestUtil() {
+  }
+
+  static ManifestFile writeDataFiles(OutputFile outputFile, PartitionSpec spec, List<DataFile> dataFiles)
+      throws IOException {
+    ManifestWriter<DataFile> writer = ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID);
+
+    try (ManifestWriter<DataFile> closeableWriter = writer) {
+      closeableWriter.addAll(dataFiles);
+    }
+
+    return writer.toManifestFile();
+  }
+
+  static List<DataFile> readDataFiles(ManifestFile manifestFile, FileIO io) throws IOException {
+    try (CloseableIterable<DataFile> dataFiles = ManifestFiles.read(manifestFile, io)) {
+      return Lists.newArrayList(dataFiles);
+    }
+  }
+
+  static ManifestOutputFileFactory createOutputFileFactory(Table table, String flinkJobId, int subTaskId,
+                                                           long attemptNumber) {
+    TableOperations ops = ((HasTableOperations) table).operations();
+    return new ManifestOutputFileFactory(ops, table.io(), table.properties(), flinkJobId, subTaskId, attemptNumber);
+  }
+
+  static DeltaManifests writeCompletedFiles(WriteResult result,
+                                            Supplier<OutputFile> outputFileSupplier,
+                                            PartitionSpec spec) throws IOException {
+
+    ManifestFile dataManifest = null;
+    ManifestFile deleteManifest = null;
+
+    // Write the completed data files into a newly created data manifest file.
+    if (result.dataFiles() != null && result.dataFiles().length > 0) {
+      dataManifest = writeDataFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles()));
+    }
+
+    // Write the completed delete files into a newly created delete manifest file.
+    if (result.deleteFiles() != null && result.deleteFiles().length > 0) {
+      OutputFile deleteManifestFile = outputFileSupplier.get();
+
+      ManifestWriter<DeleteFile> deleteManifestWriter = ManifestFiles.writeDeleteManifest(FORMAT_V2, spec,
+          deleteManifestFile, DUMMY_SNAPSHOT_ID);
+      try (ManifestWriter<DeleteFile> writer = deleteManifestWriter) {
+        for (DeleteFile deleteFile : result.deleteFiles()) {
+          writer.add(deleteFile);
+        }
+      }
+
+      deleteManifest = deleteManifestWriter.toManifestFile();
+    }
+
+    return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles());
+  }
+
+  static WriteResult readCompletedFiles(DeltaManifests deltaManifests, FileIO io) throws IOException {
+    WriteResult.Builder builder = WriteResult.builder();
+
+    // Read the completed data files from persisted data manifest file.
+    if (deltaManifests.dataManifest() != null) {
+      builder.addDataFiles(readDataFiles(deltaManifests.dataManifest(), io));
+    }
+
+    // Read the completed delete files from persisted delete manifests file.
+    if (deltaManifests.deleteManifest() != null) {
+      try (CloseableIterable<DeleteFile> deleteFiles = ManifestFiles
+          .readDeleteManifest(deltaManifests.deleteManifest(), io, null)) {
+        builder.addDeleteFiles(deleteFiles);
+      }
+    }
+
+    return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles())
+        .build();
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
new file mode 100644
index 0000000..867f97d
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
@@ -0,0 +1,478 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.function.Function;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.UPSERT_ENABLED;
+import static org.apache.iceberg.TableProperties.UPSERT_ENABLED_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+  private static final Logger LOG = LoggerFactory.getLogger(FlinkSink.class);
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    return new Builder().forMapperOutputType(input, mapper, outputType);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private Function<String, DataStream<RowData>> inputCreator = null;
+    private TableLoader tableLoader;
+    private Table table;
+    private TableSchema tableSchema;
+    private boolean overwrite = false;
+    private DistributionMode distributionMode = null;
+    private Integer writeParallelism = null;
+    private boolean upsert = false;
+    private List<String> equalityFieldColumns = null;
+    private String uidPrefix = null;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.inputCreator = ignored -> newRowDataInput;
+      return this;
+    }
+
+    private <T> Builder forMapperOutputType(DataStream<T> input,
+                                            MapFunction<T, RowData> mapper,
+                                            TypeInformation<RowData> outputType) {
+      this.inputCreator = newUidPrefix -> {
+        if (newUidPrefix != null) {
+          return input.map(mapper, outputType)
+              .name(operatorName(newUidPrefix))
+              .uid(newUidPrefix + "-mapper");
+        } else {
+          return input.map(mapper, outputType);
+        }
+      };
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    public Builder overwrite(boolean newOverwrite) {
+      this.overwrite = newOverwrite;
+      return this;
+    }
+
+    /**
+     * Configure the write {@link DistributionMode} that the flink sink will use. Currently, flink support
+     * {@link DistributionMode#NONE} and {@link DistributionMode#HASH}.
+     *
+     * @param mode to specify the write distribution mode.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder distributionMode(DistributionMode mode) {
+      Preconditions.checkArgument(!DistributionMode.RANGE.equals(mode),
+          "Flink does not support 'range' write distribution mode now.");
+      this.distributionMode = mode;
+      return this;
+    }
+
+    /**
+     * Configuring the write parallel number for iceberg stream writer.
+     *
+     * @param newWriteParallelism the number of parallel iceberg stream writer.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder writeParallelism(int newWriteParallelism) {
+      this.writeParallelism = newWriteParallelism;
+      return this;
+    }
+
+    /**
+     * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which means it will
+     * DELETE the old records and then INSERT the new records. In partitioned table, the partition fields should be
+     * a subset of equality fields, otherwise the old row that located in partition-A could not be deleted by the
+     * new row that located in partition-B.
+     *
+     * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder upsert(boolean enabled) {
+      this.upsert = enabled;
+      return this;
+    }
+
+    /**
+     * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events.
+     *
+     * @param columns defines the iceberg table's key.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder equalityFieldColumns(List<String> columns) {
+      this.equalityFieldColumns = columns;
+      return this;
+    }
+
+    /**
+     * Set the uid prefix for FlinkSink operators. Note that FlinkSink internally consists of multiple operators (like
+     * writer, committer, dummy sink etc.) Actually operator uid will be appended with a suffix like "uidPrefix-writer".
+     * <br><br>
+     * If provided, this prefix is also applied to operator names.
+     * <br><br>
+     * Flink auto generates operator uid if not set explicitly. It is a recommended
+     * <a href="https://ci.apache.org/projects/flink/flink-docs-master/docs/ops/production_ready/">
+     * best-practice to set uid for all operators</a> before deploying to production. Flink has an option to {@code
+     * pipeline.auto-generate-uid=false} to disable auto-generation and force explicit setting of all operator uid.
+     * <br><br>
+     * Be careful with setting this for an existing job, because now we are changing the operator uid from an
+     * auto-generated one to this new value. When deploying the change with a checkpoint, Flink won't be able to restore
+     * the previous Flink sink operator state (more specifically the committer operator state). You need to use {@code
+     * --allowNonRestoredState} to ignore the previous sink state. During restore Flink sink state is used to check if
+     * last commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss if the
+     * Iceberg commit failed in the last completed checkpoint.
+     *
+     * @param newPrefix prefix for Flink sink operator uid and name
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder uidPrefix(String newPrefix) {
+      this.uidPrefix = newPrefix;
+      return this;
+    }
+
+    private <T> DataStreamSink<T> chainIcebergOperators() {
+      Preconditions.checkArgument(inputCreator != null,
+          "Please use forRowData() or forMapperOutputType() to initialize the input DataStream.");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+
+      DataStream<RowData> rowDataInput = inputCreator.apply(uidPrefix);
+
+      if (table == null) {
+        tableLoader.open();
+        try (TableLoader loader = tableLoader) {
+          this.table = loader.loadTable();
+        } catch (IOException e) {
+          throw new UncheckedIOException("Failed to load iceberg table from table loader: " + tableLoader, e);
+        }
+      }
+
+      // Convert the requested flink table schema to flink row type.
+      RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema);
+
+      // Distribute the records from input data stream based on the write.distribution-mode.
+      DataStream<RowData> distributeStream = distributeDataStream(
+          rowDataInput, table.properties(), table.spec(), table.schema(), flinkRowType);
+
+      // Add parallel writers that append rows to files
+      SingleOutputStreamOperator<WriteResult> writerStream = appendWriter(distributeStream, flinkRowType);
+
+      // Add single-parallelism committer that commits files
+      // after successful checkpoint or end of input
+      SingleOutputStreamOperator<Void> committerStream = appendCommitter(writerStream);
+
+      // Add dummy discard sink
+      return appendDummySink(committerStream);
+    }
+
+    /**
+     * Append the iceberg sink operators to write records to iceberg table.
+     *
+     * @return {@link DataStreamSink} for sink.
+     * @deprecated this will be removed in 0.14.0; use {@link #append()} because its returned {@link DataStreamSink}
+     * has a more correct data type.
+     */
+    @Deprecated
+    public DataStreamSink<RowData> build() {
+      return chainIcebergOperators();
+    }
+
+    /**
+     * Append the iceberg sink operators to write records to iceberg table.
+     *
+     * @return {@link DataStreamSink} for sink.
+     */
+    public DataStreamSink<Void> append() {
+      return chainIcebergOperators();
+    }
+
+    private String operatorName(String suffix) {
+      return uidPrefix != null ? uidPrefix + "-" + suffix : suffix;
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> DataStreamSink<T> appendDummySink(SingleOutputStreamOperator<Void> committerStream) {
+      DataStreamSink<T> resultStream = committerStream
+          .addSink(new DiscardingSink())
+          .name(operatorName(String.format("IcebergSink %s", this.table.name())))
+          .setParallelism(1);
+      if (uidPrefix != null) {
+        resultStream = resultStream.uid(uidPrefix + "-dummysink");
+      }
+      return resultStream;
+    }
+
+    private SingleOutputStreamOperator<Void> appendCommitter(SingleOutputStreamOperator<WriteResult> writerStream) {
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, overwrite);
+      SingleOutputStreamOperator<Void> committerStream = writerStream
+          .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter)
+          .setParallelism(1)
+          .setMaxParallelism(1);
+      if (uidPrefix != null) {
+        committerStream = committerStream.uid(uidPrefix + "-committer");
+      }
+      return committerStream;
+    }
+
+    private SingleOutputStreamOperator<WriteResult> appendWriter(DataStream<RowData> input, RowType flinkRowType) {
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList();
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldIds.add(field.fieldId());
+        }
+      }
+
+      // Fallback to use upsert mode parsed from table properties if don't specify in job level.
+      boolean upsertMode = upsert || PropertyUtil.propertyAsBoolean(table.properties(),
+          UPSERT_ENABLED, UPSERT_ENABLED_DEFAULT);
+
+      // Validate the equality fields and partition fields if we enable the upsert mode.
+      if (upsertMode) {
+        Preconditions.checkState(!overwrite,
+            "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream.");
+        Preconditions.checkState(!equalityFieldIds.isEmpty(),
+            "Equality field columns shouldn't be empty when configuring to use UPSERT data stream.");
+        if (!table.spec().isUnpartitioned()) {
+          for (PartitionField partitionField : table.spec().fields()) {
+            Preconditions.checkState(equalityFieldIds.contains(partitionField.sourceId()),
+                "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'",
+                partitionField, equalityFieldColumns);
+          }
+        }
+      }
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, flinkRowType, equalityFieldIds, upsertMode);
+
+      int parallelism = writeParallelism == null ? input.getParallelism() : writeParallelism;
+      SingleOutputStreamOperator<WriteResult> writerStream = input
+          .transform(operatorName(ICEBERG_STREAM_WRITER_NAME), TypeInformation.of(WriteResult.class), streamWriter)
+          .setParallelism(parallelism);
+      if (uidPrefix != null) {
+        writerStream = writerStream.uid(uidPrefix + "-writer");
+      }
+      return writerStream;
+    }
+
+    private DataStream<RowData> distributeDataStream(DataStream<RowData> input,
+                                                     Map<String, String> properties,
+                                                     PartitionSpec partitionSpec,
+                                                     Schema iSchema,
+                                                     RowType flinkRowType) {
+      DistributionMode writeMode;
+      if (distributionMode == null) {
+        // Fallback to use distribution mode parsed from table properties if don't specify in job level.
+        String modeName = PropertyUtil.propertyAsString(properties,
+            WRITE_DISTRIBUTION_MODE,
+            WRITE_DISTRIBUTION_MODE_DEFAULT);
+
+        writeMode = DistributionMode.fromName(modeName);
+      } else {
+        writeMode = distributionMode;
+      }
+
+      switch (writeMode) {
+        case NONE:
+          return input;
+
+        case HASH:
+          if (partitionSpec.isUnpartitioned()) {
+            return input;
+          } else {
+            return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+          }
+
+        case RANGE:
+          LOG.warn("Fallback to use 'none' distribution mode, because {}={} is not supported in flink now",
+              WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName());
+          return input;
+
+        default:
+          throw new RuntimeException("Unrecognized write.distribution-mode: " + writeMode);
+      }
+    }
+  }
+
+  static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) {
+    if (requestedSchema != null) {
+      // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing iceberg schema.
+      Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema);
+      TypeUtil.validateWriteSchema(schema, writeSchema, true, true);
+
+      // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will be promoted to
+      // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 'byte'), we will
+      // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here we must use flink
+      // schema.
+      return (RowType) requestedSchema.toRowDataType().getLogicalType();
+    } else {
+      return FlinkSchemaUtil.convert(schema);
+    }
+  }
+
+  static IcebergStreamWriter<RowData> createStreamWriter(Table table,
+                                                         RowType flinkRowType,
+                                                         List<Integer> equalityFieldIds,
+                                                         boolean upsert) {
+    Preconditions.checkArgument(table != null, "Iceberg table should't be null");
+    Map<String, String> props = table.properties();
+    long targetFileSize = getTargetFileSizeBytes(props);
+    FileFormat fileFormat = getFileFormat(props);
+
+    Table serializableTable = SerializableTable.copyOf(table);
+    TaskWriterFactory<RowData> taskWriterFactory = new RowDataTaskWriterFactory(
+        serializableTable, flinkRowType, targetFileSize,
+        fileFormat, equalityFieldIds, upsert);
+
+    return new IcebergStreamWriter<>(table.name(), taskWriterFactory);
+  }
+
+  private static FileFormat getFileFormat(Map<String, String> properties) {
+    String formatString = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+    return FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH));
+  }
+
+  private static long getTargetFileSizeBytes(Map<String, String> properties) {
+    return PropertyUtil.propertyAsLong(properties,
+        WRITE_TARGET_FILE_SIZE_BYTES,
+        WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT);
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
new file mode 100644
index 0000000..8f8bdad
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
@@ -0,0 +1,383 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.SortedMap;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ReplacePartitions;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class IcebergFilesCommitter extends AbstractStreamOperator<Void>
+    implements OneInputStreamOperator<WriteResult, Void>, BoundedOneInput {
+
+  private static final long serialVersionUID = 1L;
+  private static final long INITIAL_CHECKPOINT_ID = -1L;
+  private static final byte[] EMPTY_MANIFEST_DATA = new byte[0];
+
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergFilesCommitter.class);
+  private static final String FLINK_JOB_ID = "flink.job-id";
+
+  // The max checkpoint id we've committed to iceberg table. As the flink's checkpoint is always increasing, so we could
+  // correctly commit all the data files whose checkpoint id is greater than the max committed one to iceberg table, for
+  // avoiding committing the same data files twice. This id will be attached to iceberg's meta when committing the
+  // iceberg transaction.
+  private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id";
+  static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits";
+
+  // TableLoader to load iceberg table lazily.
+  private final TableLoader tableLoader;
+  private final boolean replacePartitions;
+
+  // A sorted map to maintain the completed data files for each pending checkpointId (which have not been committed
+  // to iceberg table). We need a sorted map here because there's possible that few checkpoints snapshot failed, for
+  // example: the 1st checkpoint have 2 data files <1, <file0, file1>>, the 2st checkpoint have 1 data files
+  // <2, <file3>>. Snapshot for checkpoint#1 interrupted because of network/disk failure etc, while we don't expect
+  // any data loss in iceberg table. So we keep the finished files <1, <file0, file1>> in memory and retry to commit
+  // iceberg table when the next checkpoint happen.
+  private final NavigableMap<Long, byte[]> dataFilesPerCheckpoint = Maps.newTreeMap();
+
+  // The completed files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the
+  // 'dataFilesPerCheckpoint'.
+  private final List<WriteResult> writeResultsOfCurrentCkpt = Lists.newArrayList();
+
+  // It will have an unique identifier for one job.
+  private transient String flinkJobId;
+  private transient Table table;
+  private transient ManifestOutputFileFactory manifestOutputFileFactory;
+  private transient long maxCommittedCheckpointId;
+  private transient int continuousEmptyCheckpoints;
+  private transient int maxContinuousEmptyCommits;
+  // There're two cases that we restore from flink checkpoints: the first case is restoring from snapshot created by the
+  // same flink job; another case is restoring from snapshot created by another different job. For the second case, we
+  // need to maintain the old flink job's id in flink state backend to find the max-committed-checkpoint-id when
+  // traversing iceberg table's snapshots.
+  private static final ListStateDescriptor<String> JOB_ID_DESCRIPTOR = new ListStateDescriptor<>(
+      "iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO);
+  private transient ListState<String> jobIdState;
+  // All pending checkpoints states for this function.
+  private static final ListStateDescriptor<SortedMap<Long, byte[]>> STATE_DESCRIPTOR = buildStateDescriptor();
+  private transient ListState<SortedMap<Long, byte[]>> checkpointsState;
+
+  IcebergFilesCommitter(TableLoader tableLoader, boolean replacePartitions) {
+    this.tableLoader = tableLoader;
+    this.replacePartitions = replacePartitions;
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString();
+
+    // Open the table loader and load the table.
+    this.tableLoader.open();
+    this.table = tableLoader.loadTable();
+
+    maxContinuousEmptyCommits = PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10);
+    Preconditions.checkArgument(maxContinuousEmptyCommits > 0,
+        MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive");
+
+    int subTaskId = getRuntimeContext().getIndexOfThisSubtask();
+    int attemptId = getRuntimeContext().getAttemptNumber();
+    this.manifestOutputFileFactory = FlinkManifestUtil.createOutputFileFactory(table, flinkJobId, subTaskId, attemptId);
+    this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR);
+    this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR);
+    if (context.isRestored()) {
+      String restoredFlinkJobId = jobIdState.get().iterator().next();
+      Preconditions.checkState(!Strings.isNullOrEmpty(restoredFlinkJobId),
+          "Flink job id parsed from checkpoint snapshot shouldn't be null or empty");
+
+      // Since flink's checkpoint id will start from the max-committed-checkpoint-id + 1 in the new flink job even if
+      // it's restored from a snapshot created by another different flink job, so it's safe to assign the max committed
+      // checkpoint id from restored flink job to the current flink job.
+      this.maxCommittedCheckpointId = getMaxCommittedCheckpointId(table, restoredFlinkJobId);
+
+      NavigableMap<Long, byte[]> uncommittedDataFiles = Maps
+          .newTreeMap(checkpointsState.get().iterator().next())
+          .tailMap(maxCommittedCheckpointId, false);
+      if (!uncommittedDataFiles.isEmpty()) {
+        // Committed all uncommitted data files from the old flink job to iceberg table.
+        long maxUncommittedCheckpointId = uncommittedDataFiles.lastKey();
+        commitUpToCheckpoint(uncommittedDataFiles, restoredFlinkJobId, maxUncommittedCheckpointId);
+      }
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {
+    super.snapshotState(context);
+    long checkpointId = context.getCheckpointId();
+    LOG.info("Start to flush snapshot state to state backend, table: {}, checkpointId: {}", table, checkpointId);
+
+    // Update the checkpoint state.
+    dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId));
+    // Reset the snapshot state to the latest state.
+    checkpointsState.clear();
+    checkpointsState.add(dataFilesPerCheckpoint);
+
+    jobIdState.clear();
+    jobIdState.add(flinkJobId);
+
+    // Clear the local buffer for current checkpoint.
+    writeResultsOfCurrentCkpt.clear();
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // It's possible that we have the following events:
+    //   1. snapshotState(ckpId);
+    //   2. snapshotState(ckpId+1);
+    //   3. notifyCheckpointComplete(ckpId+1);
+    //   4. notifyCheckpointComplete(ckpId);
+    // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all the files,
+    // Besides, we need to maintain the max-committed-checkpoint-id to be increasing.
+    if (checkpointId > maxCommittedCheckpointId) {
+      commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, checkpointId);
+      this.maxCommittedCheckpointId = checkpointId;
+    }
+  }
+
+  private void commitUpToCheckpoint(NavigableMap<Long, byte[]> deltaManifestsMap,
+                                    String newFlinkJobId,
+                                    long checkpointId) throws IOException {
+    NavigableMap<Long, byte[]> pendingMap = deltaManifestsMap.headMap(checkpointId, true);
+    List<ManifestFile> manifests = Lists.newArrayList();
+    NavigableMap<Long, WriteResult> pendingResults = Maps.newTreeMap();
+    for (Map.Entry<Long, byte[]> e : pendingMap.entrySet()) {
+      if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue())) {
+        // Skip the empty flink manifest.
+        continue;
+      }
+
+      DeltaManifests deltaManifests = SimpleVersionedSerialization
+          .readVersionAndDeSerialize(DeltaManifestsSerializer.INSTANCE, e.getValue());
+      pendingResults.put(e.getKey(), FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io()));
+      manifests.addAll(deltaManifests.manifests());
+    }
+
+    int totalFiles = pendingResults.values().stream()
+        .mapToInt(r -> r.dataFiles().length + r.deleteFiles().length).sum();
+    continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0;
+    if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) {
+      if (replacePartitions) {
+        replacePartitions(pendingResults, newFlinkJobId, checkpointId);
+      } else {
+        commitDeltaTxn(pendingResults, newFlinkJobId, checkpointId);
+      }
+      continuousEmptyCheckpoints = 0;
+    }
+    pendingMap.clear();
+
+    // Delete the committed manifests.
+    for (ManifestFile manifest : manifests) {
+      try {
+        table.io().deleteFile(manifest.path());
+      } catch (Exception e) {
+        // The flink manifests cleaning failure shouldn't abort the completed checkpoint.
+        String details = MoreObjects.toStringHelper(this)
+            .add("flinkJobId", newFlinkJobId)
+            .add("checkpointId", checkpointId)
+            .add("manifestPath", manifest.path())
+            .toString();
+        LOG.warn("The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}",
+            details, e);
+      }
+    }
+  }
+
+  private void replacePartitions(NavigableMap<Long, WriteResult> pendingResults, String newFlinkJobId,
+                                 long checkpointId) {
+    // Partition overwrite does not support delete files.
+    int deleteFilesNum = pendingResults.values().stream().mapToInt(r -> r.deleteFiles().length).sum();
+    Preconditions.checkState(deleteFilesNum == 0, "Cannot overwrite partitions with delete files.");
+
+    // Commit the overwrite transaction.
+    ReplacePartitions dynamicOverwrite = table.newReplacePartitions();
+
+    int numFiles = 0;
+    for (WriteResult result : pendingResults.values()) {
+      Preconditions.checkState(result.referencedDataFiles().length == 0, "Should have no referenced data files.");
+
+      numFiles += result.dataFiles().length;
+      Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile);
+    }
+
+    commitOperation(dynamicOverwrite, numFiles, 0, "dynamic partition overwrite", newFlinkJobId, checkpointId);
+  }
+
+  private void commitDeltaTxn(NavigableMap<Long, WriteResult> pendingResults, String newFlinkJobId, long checkpointId) {
+    int deleteFilesNum = pendingResults.values().stream().mapToInt(r -> r.deleteFiles().length).sum();
+
+    if (deleteFilesNum == 0) {
+      // To be compatible with iceberg format V1.
+      AppendFiles appendFiles = table.newAppend();
+
+      int numFiles = 0;
+      for (WriteResult result : pendingResults.values()) {
+        Preconditions.checkState(result.referencedDataFiles().length == 0, "Should have no referenced data files.");
+
+        numFiles += result.dataFiles().length;
+        Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile);
+      }
+
+      commitOperation(appendFiles, numFiles, 0, "append", newFlinkJobId, checkpointId);
+    } else {
+      // To be compatible with iceberg format V2.
+      for (Map.Entry<Long, WriteResult> e : pendingResults.entrySet()) {
+        // We don't commit the merged result into a single transaction because for the sequential transaction txn1 and
+        // txn2, the equality-delete files of txn2 are required to be applied to data files from txn1. Committing the
+        // merged one will lead to the incorrect delete semantic.
+        WriteResult result = e.getValue();
+
+        // Row delta validations are not needed for streaming changes that write equality deletes. Equality deletes
+        // are applied to data in all previous sequence numbers, so retries may push deletes further in the future,
+        // but do not affect correctness. Position deletes committed to the table in this path are used only to delete
+        // rows from data files that are being added in this commit. There is no way for data files added along with
+        // the delete files to be concurrently removed, so there is no need to validate the files referenced by the
+        // position delete files that are being committed.
+        RowDelta rowDelta = table.newRowDelta();
+
+        int numDataFiles = result.dataFiles().length;
+        Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows);
+
+        int numDeleteFiles = result.deleteFiles().length;
+        Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes);
+
+        commitOperation(rowDelta, numDataFiles, numDeleteFiles, "rowDelta", newFlinkJobId, e.getKey());
+      }
+    }
+  }
+
+  private void commitOperation(SnapshotUpdate<?> operation, int numDataFiles, int numDeleteFiles, String description,
+                               String newFlinkJobId, long checkpointId) {
+    LOG.info("Committing {} with {} data files and {} delete files to table {}", description, numDataFiles,
+        numDeleteFiles, table);
+    operation.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId));
+    operation.set(FLINK_JOB_ID, newFlinkJobId);
+
+    long start = System.currentTimeMillis();
+    operation.commit(); // abort is automatically called if this fails.
+    long duration = System.currentTimeMillis() - start;
+    LOG.info("Committed in {} ms", duration);
+  }
+
+  @Override
+  public void processElement(StreamRecord<WriteResult> element) {
+    this.writeResultsOfCurrentCkpt.add(element.getValue());
+  }
+
+  @Override
+  public void endInput() throws IOException {
+    // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly.
+    long currentCheckpointId = Long.MAX_VALUE;
+    dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId));
+    writeResultsOfCurrentCkpt.clear();
+
+    commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, currentCheckpointId);
+  }
+
+  /**
+   * Write all the complete data files to a newly created manifest file and return the manifest's avro serialized bytes.
+   */
+  private byte[] writeToManifest(long checkpointId) throws IOException {
+    if (writeResultsOfCurrentCkpt.isEmpty()) {
+      return EMPTY_MANIFEST_DATA;
+    }
+
+    WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build();
+    DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles(result,
+        () -> manifestOutputFileFactory.create(checkpointId), table.spec());
+
+    return SimpleVersionedSerialization.writeVersionAndSerialize(DeltaManifestsSerializer.INSTANCE, deltaManifests);
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (tableLoader != null) {
+      tableLoader.close();
+    }
+  }
+
+  private static ListStateDescriptor<SortedMap<Long, byte[]>> buildStateDescriptor() {
+    Comparator<Long> longComparator = Comparators.forType(Types.LongType.get());
+    // Construct a SortedMapTypeInfo.
+    SortedMapTypeInfo<Long, byte[]> sortedMapTypeInfo = new SortedMapTypeInfo<>(
+        BasicTypeInfo.LONG_TYPE_INFO, PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, longComparator
+    );
+    return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo);
+  }
+
+  static long getMaxCommittedCheckpointId(Table table, String flinkJobId) {
+    Snapshot snapshot = table.currentSnapshot();
+    long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID;
+
+    while (snapshot != null) {
+      Map<String, String> summary = snapshot.summary();
+      String snapshotFlinkJobId = summary.get(FLINK_JOB_ID);
+      if (flinkJobId.equals(snapshotFlinkJobId)) {
+        String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID);
+        if (value != null) {
+          lastCommittedCheckpointId = Long.parseLong(value);
+          break;
+        }
+      }
+      Long parentSnapshotId = snapshot.parentId();
+      snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null;
+    }
+
+    return lastCommittedCheckpointId;
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java
new file mode 100644
index 0000000..cc8e6ce
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java
@@ -0,0 +1,103 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.io.TaskWriter;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+class IcebergStreamWriter<T> extends AbstractStreamOperator<WriteResult>
+    implements OneInputStreamOperator<T, WriteResult>, BoundedOneInput {
+
+  private static final long serialVersionUID = 1L;
+
+  private final String fullTableName;
+  private final TaskWriterFactory<T> taskWriterFactory;
+
+  private transient TaskWriter<T> writer;
+  private transient int subTaskId;
+  private transient int attemptId;
+
+  IcebergStreamWriter(String fullTableName, TaskWriterFactory<T> taskWriterFactory) {
+    this.fullTableName = fullTableName;
+    this.taskWriterFactory = taskWriterFactory;
+    setChainingStrategy(ChainingStrategy.ALWAYS);
+  }
+
+  @Override
+  public void open() {
+    this.subTaskId = getRuntimeContext().getIndexOfThisSubtask();
+    this.attemptId = getRuntimeContext().getAttemptNumber();
+
+    // Initialize the task writer factory.
+    this.taskWriterFactory.initialize(subTaskId, attemptId);
+
+    // Initialize the task writer.
+    this.writer = taskWriterFactory.create();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    // close all open files and emit files to downstream committer operator
+    emit(writer.complete());
+
+    this.writer = taskWriterFactory.create();
+  }
+
+  @Override
+  public void processElement(StreamRecord<T> element) throws Exception {
+    writer.write(element.getValue());
+  }
+
+  @Override
+  public void close() throws Exception {
+    super.close();
+    if (writer != null) {
+      writer.close();
+      writer = null;
+    }
+  }
+
+  @Override
+  public void endInput() throws IOException {
+    // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the remaining
+    // completed files to downstream before closing the writer so that we won't miss any of them.
+    emit(writer.complete());
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("table_name", fullTableName)
+        .add("subtask_id", subTaskId)
+        .add("attempt_id", attemptId)
+        .toString();
+  }
+
+  private void emit(WriteResult result) {
+    output.collect(new StreamRecord<>(result));
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java
new file mode 100644
index 0000000..fca8608
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java
@@ -0,0 +1,78 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+
+class ManifestOutputFileFactory {
+  // Users could define their own flink manifests directory by setting this value in table properties.
+  static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location";
+
+  private final TableOperations ops;
+  private final FileIO io;
+  private final Map<String, String> props;
+  private final String flinkJobId;
+  private final int subTaskId;
+  private final long attemptNumber;
+  private final AtomicInteger fileCount = new AtomicInteger(0);
+
+  ManifestOutputFileFactory(TableOperations ops, FileIO io, Map<String, String> props,
+                            String flinkJobId, int subTaskId, long attemptNumber) {
+    this.ops = ops;
+    this.io = io;
+    this.props = props;
+    this.flinkJobId = flinkJobId;
+    this.subTaskId = subTaskId;
+    this.attemptNumber = attemptNumber;
+  }
+
+  private String generatePath(long checkpointId) {
+    return FileFormat.AVRO.addExtension(String.format("%s-%05d-%d-%d-%05d", flinkJobId, subTaskId,
+        attemptNumber, checkpointId, fileCount.incrementAndGet()));
+  }
+
+  OutputFile create(long checkpointId) {
+    String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION);
+
+    String newManifestFullPath;
+    if (Strings.isNullOrEmpty(flinkManifestDir)) {
+      // User don't specify any flink manifest directory, so just use the default metadata path.
+      newManifestFullPath = ops.metadataFileLocation(generatePath(checkpointId));
+    } else {
+      newManifestFullPath = String.format("%s/%s", stripTrailingSlash(flinkManifestDir), generatePath(checkpointId));
+    }
+
+    return io.newOutputFile(newManifestFullPath);
+  }
+
+  private static String stripTrailingSlash(String path) {
+    String result = path;
+    while (result.endsWith("/")) {
+      result = result.substring(0, result.length() - 1);
+    }
+    return result;
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java
new file mode 100644
index 0000000..598df09
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+
+/**
+ * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be wrote by only one
+ * task. That will reduce lots of small files in partitioned fanout write policy for {@link FlinkSink}.
+ */
+class PartitionKeySelector implements KeySelector<RowData, String> {
+
+  private final Schema schema;
+  private final PartitionKey partitionKey;
+  private final RowType flinkSchema;
+
+  private transient RowDataWrapper rowDataWrapper;
+
+  PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) {
+    this.schema = schema;
+    this.partitionKey = new PartitionKey(spec, schema);
+    this.flinkSchema = flinkSchema;
+  }
+
+  /**
+   * Construct the {@link RowDataWrapper} lazily here because few members in it are not serializable. In this way, we
+   * don't have to serialize them with forcing.
+   */
+  private RowDataWrapper lazyRowDataWrapper() {
+    if (rowDataWrapper == null) {
+      rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct());
+    }
+    return rowDataWrapper;
+  }
+
+  @Override
+  public String getKey(RowData row) {
+    partitionKey.partition(lazyRowDataWrapper().wrap(row));
+    return partitionKey.toPath();
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java
new file mode 100644
index 0000000..1eee629
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java
@@ -0,0 +1,87 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Tasks;
+
+class PartitionedDeltaWriter extends BaseDeltaTaskWriter {
+
+  private final PartitionKey partitionKey;
+
+  private final Map<PartitionKey, RowDataDeltaWriter> writers = Maps.newHashMap();
+
+  PartitionedDeltaWriter(PartitionSpec spec,
+                         FileFormat format,
+                         FileAppenderFactory<RowData> appenderFactory,
+                         OutputFileFactory fileFactory,
+                         FileIO io,
+                         long targetFileSize,
+                         Schema schema,
+                         RowType flinkSchema,
+                         List<Integer> equalityFieldIds,
+                         boolean upsert) {
+    super(spec, format, appenderFactory, fileFactory, io, targetFileSize, schema, flinkSchema, equalityFieldIds,
+        upsert);
+    this.partitionKey = new PartitionKey(spec, schema);
+  }
+
+  @Override
+  RowDataDeltaWriter route(RowData row) {
+    partitionKey.partition(wrapper().wrap(row));
+
+    RowDataDeltaWriter writer = writers.get(partitionKey);
+    if (writer == null) {
+      // NOTICE: we need to copy a new partition key here, in case of messing up the keys in writers.
+      PartitionKey copiedKey = partitionKey.copy();
+      writer = new RowDataDeltaWriter(copiedKey);
+      writers.put(copiedKey, writer);
+    }
+
+    return writer;
+  }
+
+  @Override
+  public void close() {
+    try {
+      Tasks.foreach(writers.values())
+          .throwFailureWhenFinished()
+          .noRetry()
+          .run(RowDataDeltaWriter::close, IOException.class);
+
+      writers.clear();
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to close equality delta writer", e);
+    }
+  }
+}
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java
new file mode 100644
index 0000000..2849100
--- /dev/null
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import java.util.List;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.RowDataWrapper;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.io.PartitionedFanoutWriter;
+import org.apache.iceberg.io.TaskWriter;
+import org.apache.iceberg.io.UnpartitionedWriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ArrayUtil;
+
+public class RowDataTaskWriterFactory implements TaskWriterFactory<RowData> {
+  private final Table table;
+  private final Schema schema;
+  private final RowType flinkSchema;
+  private final PartitionSpec spec;
+  private final FileIO io;
+  private final long targetFileSizeBytes;
+  private final FileFormat format;
+  private final List<Integer> equalityFieldIds;
+  private final boolean upsert;
+  private final FileAppenderFactory<RowData> appenderFactory;
+
+  private transient OutputFileFactory outputFileFactory;
... 14992 lines suppressed ...