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
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import static org.apache.polaris.service.catalog.common.ExceptionUtils.alreadyExistsExceptionForTableLikeEntity;
import static org.apache.polaris.service.catalog.common.ExceptionUtils.notFoundExceptionForTableLikeEntity;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
Expand Down Expand Up @@ -1399,7 +1400,8 @@ public void renameView(RenameTableRequest request) {
catalogHandlerUtils().renameView(viewCatalog, request);
}

private @NonNull LoadTableResponse filterResponseToSnapshots(
@VisibleForTesting
@NonNull LoadTableResponse filterResponseToSnapshots(
LoadTableResponse loadTableResponse, String snapshots) {
Comment on lines +1403 to 1405
if (snapshots == null || snapshots.equalsIgnoreCase(SNAPSHOTS_ALL)) {
return loadTableResponse;
Expand All @@ -1414,8 +1416,14 @@ public void renameView(RenameTableRequest request) {
TableMetadata filteredMetadata =
metadata.removeSnapshotsIf(s -> !referencedSnapshotIds.contains(s.snapshotId()));

TableMetadata filteredMetadataWithLocation =
TableMetadata.buildFrom(filteredMetadata)
.withMetadataLocation(metadata.metadataFileLocation())
.discardChanges()
.build();
Comment on lines +1419 to +1423

return LoadTableResponse.builder()
.withTableMetadata(filteredMetadata)
.withTableMetadata(filteredMetadataWithLocation)
.addAllConfig(loadTableResponse.config())
.addAllCredentials(loadTableResponse.credentials())
.build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,13 +33,21 @@
import java.util.Optional;
import java.util.Set;
import org.apache.iceberg.BaseTable;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableMetadata;
import org.apache.iceberg.TableOperations;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.inmemory.InMemoryCatalog;
import org.apache.iceberg.rest.credentials.Credential;
import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse;
import org.apache.iceberg.rest.responses.LoadTableResponse;
import org.apache.iceberg.types.Types;
import org.apache.polaris.core.PolarisDiagnostics;
import org.apache.polaris.core.auth.PolarisAuthorizer;
import org.apache.polaris.core.auth.PolarisPrincipal;
Expand Down Expand Up @@ -296,4 +304,43 @@ void loadCredentialsFallsBackWhenEntityLocationMissing() {
assertThat(c.config()).containsExactlyInAnyOrderEntriesOf(fakeCredentials);
});
}

@Test
@SuppressWarnings("resource")
void filterResponseToSnapshotsRefsPreservesMetadataLocation() throws Exception {
Comment on lines +309 to +310
try (InMemoryCatalog catalog = new InMemoryCatalog()) {
catalog.initialize("test", Map.of());
catalog.createNamespace(NS1);
Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get()));
Table table = catalog.createTable(TABLE2, schema);
table.newFastAppend().appendFile(dataFile("file:/tmp/data/f1.parquet")).commit();
table.newFastAppend().appendFile(dataFile("file:/tmp/data/f2.parquet")).commit();

TableMetadata metadata = ((BaseTable) catalog.loadTable(TABLE2)).operations().current();
assertThat(metadata.snapshots()).hasSize(2);

LoadTableResponse response = LoadTableResponse.builder().withTableMetadata(metadata).build();
String metadataLocation = response.metadataLocation();
assertThat(metadataLocation)
.as("precondition: the unfiltered (snapshots=all) response carries a metadata-location")
.isNotNull();

LoadTableResponse filtered = newHandler().filterResponseToSnapshots(response, "refs");

assertThat(filtered.tableMetadata().snapshots())
.as("snapshots=refs must still drop the historical (non-ref) snapshot")
.hasSize(1);
assertThat(filtered.metadataLocation())
.as("metadata-location must be preserved when filtering snapshots=refs")
.isEqualTo(metadataLocation);
}
}

private static DataFile dataFile(String path) {
return DataFiles.builder(PartitionSpec.unpartitioned())
.withPath(path)
.withFileSizeInBytes(10L)
.withRecordCount(1L)
.build();
}
}