Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion api/src/main/java/org/apache/iceberg/catalog/Namespace.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,9 @@
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;

/** A namespace in a {@link Catalog}. */
public class Namespace {
public class Namespace implements java.io.Serializable {

private static final long serialVersionUID = 1L;
private static final Namespace EMPTY_NAMESPACE = new Namespace(new String[] {});
private static final Joiner DOT = Joiner.on('.');
private static final Predicate<String> CONTAINS_NULL_CHARACTER =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.iceberg.catalog;

import java.io.Serializable;
import java.util.Arrays;
import java.util.Locale;
import java.util.Objects;
Expand All @@ -26,10 +27,10 @@
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;

/** Identifies a table in iceberg catalog. */
public class TableIdentifier {
public class TableIdentifier implements Serializable {

private static final long serialVersionUID = 1L;
private static final Splitter DOT = Splitter.on('.');

private final Namespace namespace;
private final String name;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,13 +108,13 @@ class CatalogTableLoader implements TableLoader {
private static final long serialVersionUID = 1L;

private final CatalogLoader catalogLoader;
private final String identifier;
private final TableIdentifier identifier;

private transient Catalog catalog;

private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) {
this.catalogLoader = catalogLoader;
this.identifier = tableIdentifier.toString();
this.identifier = tableIdentifier;
}

@Override
Expand All @@ -130,7 +130,7 @@ public boolean isOpen() {
@Override
public Table loadTable() {
FlinkEnvironmentContext.init();
return catalog.loadTable(TableIdentifier.parse(identifier));
return catalog.loadTable(identifier);
}

@Override
Expand All @@ -145,7 +145,7 @@ public void close() throws IOException {
@Override
@SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"})
public TableLoader clone() {
return new CatalogTableLoader(catalogLoader.clone(), TableIdentifier.parse(identifier));
return new CatalogTableLoader(catalogLoader.clone(), identifier);
}

@Override
Expand Down