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
18 changes: 17 additions & 1 deletion core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.BiFunction;
import java.util.function.Function;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.PartitionSpec;
Expand All @@ -39,6 +40,8 @@
import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
import org.apache.iceberg.exceptions.NoSuchNamespaceException;
import org.apache.iceberg.hadoop.Configurable;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.view.View;
Expand Down Expand Up @@ -69,7 +72,15 @@ public RESTCatalog(Function<Map<String, String>, RESTClient> clientBuilder) {
public RESTCatalog(
SessionCatalog.SessionContext context,
Function<Map<String, String>, RESTClient> clientBuilder) {
this.sessionCatalog = new RESTSessionCatalog(clientBuilder, null);
this(context, clientBuilder, null);
}

@VisibleForTesting
RESTCatalog(
SessionCatalog.SessionContext context,
Function<Map<String, String>, RESTClient> clientBuilder,
BiFunction<SessionCatalog.SessionContext, Map<String, String>, FileIO> ioBuilder) {
this.sessionCatalog = new RESTSessionCatalog(clientBuilder, ioBuilder);
this.delegate = sessionCatalog.asCatalog(context);
this.nsDelegate = (SupportsNamespaces) delegate;
this.context = context;
Expand All @@ -82,6 +93,11 @@ public void initialize(String name, Map<String, String> props) {
sessionCatalog.initialize(name, props);
}

@VisibleForTesting
RESTSessionCatalog sessionCatalog() {
return sessionCatalog;
}

@Override
public String name() {
return sessionCatalog.name();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.apache.iceberg.io.StorageCredential;
import org.apache.iceberg.metrics.MetricsReporter;
import org.apache.iceberg.metrics.MetricsReporters;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
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.ImmutableMap;
Expand Down Expand Up @@ -674,6 +675,11 @@ public boolean updateNamespaceMetadata(
return !response.updated().isEmpty();
}

@VisibleForTesting
FileIOTracker fileIOTracker() {
return fileIOTracker;
}

@Override
public void close() throws IOException {
if (closeables != null) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.io;

import com.github.benmanes.caffeine.cache.Cache;
import org.apache.iceberg.TableOperations;

public abstract class FileIOTrackerTestUtil {
public static Cache<TableOperations, FileIO> trackerFrom(FileIOTracker tracker) {
return tracker.tracker();
}

public static void invalidate(FileIOTracker tracker, TableOperations ops) {
tracker.tracker().invalidate(ops);
tracker.tracker().cleanUp();
}
}
140 changes: 140 additions & 0 deletions core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,11 @@
import static org.mockito.Mockito.atLeastOnce;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.github.benmanes.caffeine.cache.Cache;
import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
Expand All @@ -43,19 +45,23 @@
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.function.Function;
import org.apache.hadoop.conf.Configuration;
import org.apache.http.HttpHeaders;
import org.apache.iceberg.BaseTable;
import org.apache.iceberg.BaseTransaction;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.MetadataUpdate;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableOperations;
import org.apache.iceberg.Transaction;
import org.apache.iceberg.UpdatePartitionSpec;
import org.apache.iceberg.UpdateSchema;
Expand All @@ -70,6 +76,8 @@
import org.apache.iceberg.exceptions.ServiceFailureException;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.inmemory.InMemoryCatalog;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.FileIOTrackerTestUtil;
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;
Expand Down Expand Up @@ -2884,6 +2892,138 @@ public void testETagWithRegisterTable() {
assertThat(respHeaders).containsEntry(HttpHeaders.ETAG, eTag);
}

@Test
public void testDefaultFileIOIsNotTracked() {
catalog().createNamespace(TABLE.namespace());

catalog().createTable(TABLE, SCHEMA);

catalog().loadTable(TABLE);

Cache<TableOperations, FileIO> tracker =
FileIOTrackerTestUtil.trackerFrom(catalog().sessionCatalog().fileIOTracker());

assertThat(tracker.estimatedSize()).isZero();
}

@Test
public void testCreateTableTracksFileIOFromIOBuilder() {
verifyFileIOFromIOBuilderIsTracked(catalog -> (BaseTable) catalog.createTable(TABLE, SCHEMA));
}

@Test
public void testLoadTableTracksFileIOFromIOBuilder() {
verifyFileIOFromIOBuilderIsTracked(
catalog -> {
// Use a different catalog to create the table to avoid populating the FileIOTracker in
// the catalog under test.
RESTCatalog otherCatalog = catalog(new RESTCatalogAdapter(backendCatalog));
otherCatalog.createTable(TABLE, SCHEMA);

return (BaseTable) catalog.loadTable(TABLE);
});
}

@Test
public void testRegisterTableTracksFileIOFromIOBuilder() {
verifyFileIOFromIOBuilderIsTracked(
catalog -> {
BaseTable table = (BaseTable) catalog.createTable(TABLE, SCHEMA);

TableIdentifier otherTable = TableIdentifier.of(TABLE.namespace(), "other_table");

return (BaseTable)
catalog.registerTable(
otherTable, table.operations().current().metadataFileLocation());
});
}

private void verifyFileIOFromIOBuilderIsTracked(Function<RESTCatalog, BaseTable> func) {
FileIO fileIO =
Mockito.spy(
CatalogUtil.loadFileIO("org.apache.iceberg.io.ResolvingFileIO", Map.of(), null));

BiFunction<SessionCatalog.SessionContext, Map<String, String>, FileIO> ioBuilderMock =
Mockito.mock(BiFunction.class);

// Make sure the catalog IO is different from the table IO
when(ioBuilderMock.apply(any(), any()))
.thenReturn(
CatalogUtil.loadFileIO("org.apache.iceberg.io.ResolvingFileIO", Map.of(), null),
fileIO);

RESTCatalog catalog =
new RESTCatalog(
SessionCatalog.SessionContext.createEmpty(),
config -> new RESTCatalogAdapter(backendCatalog),
ioBuilderMock);
catalog.initialize("test", Map.of());

catalog.createNamespace(TABLE.namespace());

Cache<TableOperations, FileIO> tracker =
FileIOTrackerTestUtil.trackerFrom(catalog.sessionCatalog().fileIOTracker());

assertThat(tracker.estimatedSize()).isZero();

BaseTable table = func.apply(catalog);

assertThat(tracker.asMap()).containsKeys(table.operations());

// Simulate cleanup of TableOps from the tracker
FileIOTrackerTestUtil.invalidate(catalog.sessionCatalog().fileIOTracker(), table.operations());

Awaitility.await()
.atMost(5, TimeUnit.SECONDS)
.untilAsserted(() -> Mockito.verify(fileIO).close());
}

@Test
public void testCreateTransactionTracksFileIOFromIOBuilder() {
RESTCatalog catalog =
new RESTCatalog(
SessionCatalog.SessionContext.createEmpty(),
config -> new RESTCatalogAdapter(backendCatalog),
(context, conf) ->
CatalogUtil.loadFileIO("org.apache.iceberg.io.ResolvingFileIO", Map.of(), null));
catalog.initialize("test", Map.of());

catalog.createNamespace(TABLE.namespace());

Cache<TableOperations, FileIO> tracker =
FileIOTrackerTestUtil.trackerFrom(catalog.sessionCatalog().fileIOTracker());

assertThat(tracker.estimatedSize()).isZero();

catalog.newCreateTableTransaction(TABLE, SCHEMA);

assertThat(tracker.estimatedSize()).isOne();
}

@Test
public void testReplaceTransactionTracksFileIOFromIOBuilder() {
RESTCatalog catalog =
new RESTCatalog(
SessionCatalog.SessionContext.createEmpty(),
config -> new RESTCatalogAdapter(backendCatalog),
(context, conf) ->
CatalogUtil.loadFileIO("org.apache.iceberg.io.ResolvingFileIO", Map.of(), null));
catalog.initialize("test", Map.of());

catalog.createNamespace(TABLE.namespace());

catalog.createTable(TABLE, SCHEMA);

Cache<TableOperations, FileIO> tracker =
FileIOTrackerTestUtil.trackerFrom(catalog.sessionCatalog().fileIOTracker());

assertThat(tracker.estimatedSize()).isOne();

catalog.newReplaceTableTransaction(TABLE, SCHEMA, false);

assertThat(tracker.estimatedSize()).isEqualTo(2);
}

private RESTCatalog catalogWithResponseHeaders(Map<String, String> respHeaders) {
RESTCatalogAdapter adapter =
new RESTCatalogAdapter(backendCatalog) {
Expand Down