diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html
index 6706d5c421a1..63f7adda1e0d 100644
--- a/docs/layouts/shortcodes/generated/catalog_configuration.html
+++ b/docs/layouts/shortcodes/generated/catalog_configuration.html
@@ -86,12 +86,6 @@
lock-acquire-timeout |
8 min |
diff --git a/paimon-common/src/main/java/org/apache/paimon/factories/Factory.java b/paimon-common/src/main/java/org/apache/paimon/factories/Factory.java
index b0f1ec84c170..74796879ef4b 100644
--- a/paimon-common/src/main/java/org/apache/paimon/factories/Factory.java
+++ b/paimon-common/src/main/java/org/apache/paimon/factories/Factory.java
@@ -20,7 +20,7 @@
/**
* Base interface for all kind of factories that create object instances from a list of key-value
- * pairs in Paimon's catalog, lineage.
+ * pairs in Paimon's catalog.
*
* A factory is uniquely identified by {@link Class} and {@link #identifier()}.
*
diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java b/paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java
deleted file mode 100644
index e7401a9be3b7..000000000000
--- a/paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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.paimon.lineage;
-
-import org.apache.paimon.data.Timestamp;
-
-/**
- * Data lineage entity with table lineage, barrier id and snapshot id for table source and sink
- * lineage.
- */
-public interface DataLineageEntity extends TableLineageEntity {
- long getBarrierId();
-
- long getSnapshotId();
-
- Timestamp getCreateTime();
-}
diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMeta.java b/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMeta.java
deleted file mode 100644
index 5d1c42daf6c8..000000000000
--- a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMeta.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.paimon.lineage;
-
-import org.apache.paimon.predicate.Predicate;
-
-import javax.annotation.Nullable;
-
-import java.util.Iterator;
-
-/** Metadata store will manage table lineage and data lineage information for the catalog. */
-public interface LineageMeta extends AutoCloseable {
- /**
- * Save the source table and job lineage.
- *
- * @param entity the table lineage entity
- */
- void saveSourceTableLineage(TableLineageEntity entity);
-
- /**
- * Delete the source table lineage for given job.
- *
- * @param job the job for table lineage
- */
- void deleteSourceTableLineage(String job);
-
- /**
- * Get source table and job lineages.
- *
- * @param predicate the predicate for the table lineages
- * @return the iterator for source table and job lineages
- */
- Iterator sourceTableLineages(@Nullable Predicate predicate);
-
- /**
- * Save the sink table and job lineage.
- *
- * @param entity the table lineage entity
- */
- void saveSinkTableLineage(TableLineageEntity entity);
-
- /**
- * Get sink table and job lineages.
- *
- * @param predicate the predicate for the table lineages
- * @return the iterator for sink table and job lineages
- */
- Iterator sinkTableLineages(@Nullable Predicate predicate);
-
- /**
- * Delete the sink table lineage for given job.
- *
- * @param job the job for table lineage
- */
- void deleteSinkTableLineage(String job);
-
- /**
- * Save the source table and job lineage.
- *
- * @param entity the data lineage entity
- */
- void saveSourceDataLineage(DataLineageEntity entity);
-
- /**
- * Get source data and job lineages.
- *
- * @param predicate the predicate for the table lineages
- * @return the iterator for source table and job lineages
- */
- Iterator sourceDataLineages(@Nullable Predicate predicate);
-
- /**
- * Save the sink table and job lineage.
- *
- * @param entity the data lineage entity
- */
- void saveSinkDataLineage(DataLineageEntity entity);
-
- /**
- * Get sink data and job lineages.
- *
- * @param predicate the predicate for the table lineages
- * @return the iterator for sink table and job lineages
- */
- Iterator sinkDataLineages(@Nullable Predicate predicate);
-}
diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java b/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java
deleted file mode 100644
index 11c6d3a1173c..000000000000
--- a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.paimon.lineage;
-
-import org.apache.paimon.factories.Factory;
-import org.apache.paimon.options.Options;
-
-import java.io.Serializable;
-
-/** Factory to create {@link LineageMeta}. Each factory should have a unique identifier. */
-public interface LineageMetaFactory extends Factory, Serializable {
-
- LineageMeta create(LineageMetaContext context);
-
- /**
- * Context has all options in a catalog and is used in factory to create {@link LineageMeta}.
- */
- interface LineageMetaContext {
- Options options();
- }
-}
diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java b/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java
deleted file mode 100644
index c4312c4eb080..000000000000
--- a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.paimon.lineage;
-
-import org.apache.paimon.data.Timestamp;
-
-/** Table lineage entity with database, table and job for table source and sink lineage. */
-public interface TableLineageEntity {
- String getDatabase();
-
- String getTable();
-
- String getJob();
-
- Timestamp getCreateTime();
-}
diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java b/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java
deleted file mode 100644
index ef11ee87f15c..000000000000
--- a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.paimon.lineage;
-
-import org.apache.paimon.data.Timestamp;
-
-/** Default implementation for {@link TableLineageEntity}. */
-public class TableLineageEntityImpl implements TableLineageEntity {
- private final String database;
- private final String table;
- private final String job;
- private final Timestamp timestamp;
-
- public TableLineageEntityImpl(String database, String table, String job, Timestamp timestamp) {
- this.database = database;
- this.table = table;
- this.job = job;
- this.timestamp = timestamp;
- }
-
- @Override
- public String getDatabase() {
- return database;
- }
-
- @Override
- public String getTable() {
- return table;
- }
-
- @Override
- public String getJob() {
- return job;
- }
-
- @Override
- public Timestamp getCreateTime() {
- return timestamp;
- }
-}
diff --git a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
index f69af2d59910..bb8cfae68284 100644
--- a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
+++ b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
@@ -18,8 +18,6 @@
package org.apache.paimon.options;
-import org.apache.paimon.options.description.Description;
-import org.apache.paimon.options.description.TextElement;
import org.apache.paimon.table.CatalogTableType;
import java.time.Duration;
@@ -130,26 +128,6 @@ public class CatalogOptions {
.withDescription(
"Controls the max number for snapshots per table in the catalog are cached.");
- public static final ConfigOption LINEAGE_META =
- key("lineage-meta")
- .stringType()
- .noDefaultValue()
- .withDescription(
- Description.builder()
- .text(
- "The lineage meta to store table and data lineage information.")
- .linebreak()
- .linebreak()
- .text("Possible values:")
- .linebreak()
- .list(
- TextElement.text(
- "\"jdbc\": Use standard jdbc to store table and data lineage information."))
- .list(
- TextElement.text(
- "\"custom\": You can implement LineageMetaFactory and LineageMeta to store lineage information in customized storage."))
- .build());
-
public static final ConfigOption ALLOW_UPPER_CASE =
ConfigOptions.key("allow-upper-case")
.booleanType()
diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java
index 16b76513d7ef..2b277a29b835 100644
--- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java
+++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java
@@ -24,7 +24,6 @@
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.FileStatus;
import org.apache.paimon.fs.Path;
-import org.apache.paimon.lineage.LineageMetaFactory;
import org.apache.paimon.manifest.PartitionEntry;
import org.apache.paimon.metastore.MetastoreClient;
import org.apache.paimon.operation.FileStoreCommit;
@@ -62,7 +61,6 @@
import static org.apache.paimon.CoreOptions.TYPE;
import static org.apache.paimon.CoreOptions.createCommitUser;
import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE;
-import static org.apache.paimon.options.CatalogOptions.LINEAGE_META;
import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED;
import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE;
import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
@@ -76,19 +74,14 @@ public abstract class AbstractCatalog implements Catalog {
protected final Map tableDefaultOptions;
protected final Options catalogOptions;
- @Nullable protected final LineageMetaFactory lineageMetaFactory;
-
protected AbstractCatalog(FileIO fileIO) {
this.fileIO = fileIO;
- this.lineageMetaFactory = null;
this.tableDefaultOptions = new HashMap<>();
this.catalogOptions = new Options();
}
protected AbstractCatalog(FileIO fileIO, Options options) {
this.fileIO = fileIO;
- this.lineageMetaFactory =
- findAndCreateLineageMeta(options, AbstractCatalog.class.getClassLoader());
this.tableDefaultOptions = Catalog.tableDefaultOptions(options.toMap());
this.catalogOptions = options;
}
@@ -377,27 +370,13 @@ public void alterTable(
protected abstract void alterTableImpl(Identifier identifier, List changes)
throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException;
- @Nullable
- private LineageMetaFactory findAndCreateLineageMeta(Options options, ClassLoader classLoader) {
- return options.getOptional(LINEAGE_META)
- .map(
- meta ->
- FactoryUtil.discoverFactory(
- classLoader, LineageMetaFactory.class, meta))
- .orElse(null);
- }
-
@Override
public Table getTable(Identifier identifier) throws TableNotExistException {
if (isSystemDatabase(identifier.getDatabaseName())) {
String tableName = identifier.getTableName();
Table table =
SystemTableLoader.loadGlobal(
- tableName,
- fileIO,
- this::allTablePaths,
- catalogOptions,
- lineageMetaFactory);
+ tableName, fileIO, this::allTablePaths, catalogOptions);
if (table == null) {
throw new TableNotExistException(identifier);
}
@@ -444,8 +423,7 @@ protected Table getDataOrFormatTable(Identifier identifier) throws TableNotExist
lockFactory().orElse(null),
lockContext().orElse(null),
identifier),
- metastoreClientFactory(identifier, tableMeta.schema).orElse(null),
- lineageMetaFactory));
+ metastoreClientFactory(identifier, tableMeta.schema).orElse(null)));
CoreOptions options = table.coreOptions();
if (options.type() == TableType.OBJECT_TABLE) {
String objectLocation = options.objectLocation();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java b/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java
index 9ff5f9b4f6a8..a722d9e21ada 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java
@@ -19,7 +19,6 @@
package org.apache.paimon.table;
import org.apache.paimon.catalog.Identifier;
-import org.apache.paimon.lineage.LineageMetaFactory;
import org.apache.paimon.metastore.MetastoreClient;
import org.apache.paimon.operation.Lock;
@@ -27,10 +26,7 @@
import java.io.Serializable;
-/**
- * Catalog environment in table which contains log factory, metastore client factory and lineage
- * meta.
- */
+/** Catalog environment in table which contains log factory, metastore client factory. */
public class CatalogEnvironment implements Serializable {
private static final long serialVersionUID = 1L;
@@ -39,23 +35,20 @@ public class CatalogEnvironment implements Serializable {
@Nullable private final String uuid;
private final Lock.Factory lockFactory;
@Nullable private final MetastoreClient.Factory metastoreClientFactory;
- @Nullable private final LineageMetaFactory lineageMetaFactory;
public CatalogEnvironment(
@Nullable Identifier identifier,
@Nullable String uuid,
Lock.Factory lockFactory,
- @Nullable MetastoreClient.Factory metastoreClientFactory,
- @Nullable LineageMetaFactory lineageMetaFactory) {
+ @Nullable MetastoreClient.Factory metastoreClientFactory) {
this.identifier = identifier;
this.uuid = uuid;
this.lockFactory = lockFactory;
this.metastoreClientFactory = metastoreClientFactory;
- this.lineageMetaFactory = lineageMetaFactory;
}
public static CatalogEnvironment empty() {
- return new CatalogEnvironment(null, null, Lock.emptyFactory(), null, null);
+ return new CatalogEnvironment(null, null, Lock.emptyFactory(), null);
}
@Nullable
@@ -76,9 +69,4 @@ public Lock.Factory lockFactory() {
public MetastoreClient.Factory metastoreClientFactory() {
return metastoreClientFactory;
}
-
- @Nullable
- public LineageMetaFactory lineageMetaFactory() {
- return lineageMetaFactory;
- }
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java
deleted file mode 100644
index 71efce070471..000000000000
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.paimon.table.system;
-
-import org.apache.paimon.lineage.LineageMeta;
-import org.apache.paimon.lineage.LineageMetaFactory;
-import org.apache.paimon.options.Options;
-import org.apache.paimon.table.Table;
-import org.apache.paimon.table.source.InnerTableRead;
-
-import java.util.Map;
-
-/**
- * This is a system table to display all the sink table lineages.
- *
- *
- * For example:
- * If we select * from sys.sink_table_lineage, we will get
- * database_name table_name job_name create_time
- * default test0 job1 2023-10-22 20:35:12
- * database1 test1 job1 2023-10-28 21:35:52
- * ... ... ... ...
- * We can write sql to fetch the information we need.
- *
- */
-public class SinkTableLineageTable extends TableLineageTable {
-
- public static final String SINK_TABLE_LINEAGE = "sink_table_lineage";
-
- public SinkTableLineageTable(LineageMetaFactory lineageMetaFactory, Options options) {
- super(lineageMetaFactory, options);
- }
-
- @Override
- public InnerTableRead newRead() {
- return new TableLineageRead(lineageMetaFactory, options, LineageMeta::sinkTableLineages);
- }
-
- @Override
- public String name() {
- return SINK_TABLE_LINEAGE;
- }
-
- @Override
- public Table copy(Map dynamicOptions) {
- return new SinkTableLineageTable(lineageMetaFactory, options);
- }
-}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java
deleted file mode 100644
index 5d9904fa6675..000000000000
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.paimon.table.system;
-
-import org.apache.paimon.lineage.LineageMeta;
-import org.apache.paimon.lineage.LineageMetaFactory;
-import org.apache.paimon.options.Options;
-import org.apache.paimon.table.Table;
-import org.apache.paimon.table.source.InnerTableRead;
-
-import java.util.Map;
-
-/**
- * This is a system table to display all the source table lineages.
- *
- *
- * For example:
- * If we select * from sys.source_table_lineage, we will get
- * database_name table_name job_name create_time
- * default test0 job1 2023-10-22 20:35:12
- * database1 test1 job1 2023-10-28 21:35:52
- * ... ... ... ...
- * We can write sql to fetch the information we need.
- *
- */
-public class SourceTableLineageTable extends TableLineageTable {
-
- public static final String SOURCE_TABLE_LINEAGE = "source_table_lineage";
-
- public SourceTableLineageTable(LineageMetaFactory lineageMetaFactory, Options options) {
- super(lineageMetaFactory, options);
- }
-
- @Override
- public InnerTableRead newRead() {
- return new TableLineageRead(lineageMetaFactory, options, LineageMeta::sourceTableLineages);
- }
-
- @Override
- public String name() {
- return SOURCE_TABLE_LINEAGE;
- }
-
- @Override
- public Table copy(Map dynamicOptions) {
- return new SourceTableLineageTable(lineageMetaFactory, options);
- }
-}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java
index 3d5b211316ec..763e4d121673 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java
@@ -20,7 +20,6 @@
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.Path;
-import org.apache.paimon.lineage.LineageMetaFactory;
import org.apache.paimon.options.Options;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.Table;
@@ -37,7 +36,6 @@
import java.util.function.Function;
import java.util.function.Supplier;
-import static org.apache.paimon.options.CatalogOptions.LINEAGE_META;
import static org.apache.paimon.table.system.AggregationFieldsTable.AGGREGATION_FIELDS;
import static org.apache.paimon.table.system.AllTableOptionsTable.ALL_TABLE_OPTIONS;
import static org.apache.paimon.table.system.AuditLogTable.AUDIT_LOG;
@@ -52,12 +50,9 @@
import static org.apache.paimon.table.system.PartitionsTable.PARTITIONS;
import static org.apache.paimon.table.system.ReadOptimizedTable.READ_OPTIMIZED;
import static org.apache.paimon.table.system.SchemasTable.SCHEMAS;
-import static org.apache.paimon.table.system.SinkTableLineageTable.SINK_TABLE_LINEAGE;
import static org.apache.paimon.table.system.SnapshotsTable.SNAPSHOTS;
-import static org.apache.paimon.table.system.SourceTableLineageTable.SOURCE_TABLE_LINEAGE;
import static org.apache.paimon.table.system.StatisticTable.STATISTICS;
import static org.apache.paimon.table.system.TagsTable.TAGS;
-import static org.apache.paimon.utils.Preconditions.checkNotNull;
/** Loader to load system {@link Table}s. */
public class SystemTableLoader {
@@ -95,38 +90,18 @@ public static Table loadGlobal(
String tableName,
FileIO fileIO,
Supplier