From 5f03a399d716899981d0def1ce0273ea656b29e7 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 17 May 2023 09:56:39 +0200 Subject: [PATCH 01/14] Core: Add remaining View APIs and support for InMemoryCatalog --- .../iceberg/inmemory/InMemoryCatalog.java | 140 ++- .../view/BaseMetastoreViewCatalog.java | 190 ++++ .../org/apache/iceberg/view/BaseView.java | 89 ++ .../iceberg/view/BaseViewOperations.java | 220 ++++ .../apache/iceberg/view/PropertiesUpdate.java | 102 ++ .../apache/iceberg/view/ViewOperations.java | 59 ++ .../org/apache/iceberg/view/ViewUtil.java | 29 + .../iceberg/view/ViewVersionReplace.java | 132 +++ .../inmemory/TestInMemoryViewCatalog.java | 49 + .../apache/iceberg/view/ViewCatalogTests.java | 943 ++++++++++++++++++ 10 files changed, 1948 insertions(+), 5 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java create mode 100644 core/src/main/java/org/apache/iceberg/view/BaseView.java create mode 100644 core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java create mode 100644 core/src/main/java/org/apache/iceberg/view/PropertiesUpdate.java create mode 100644 core/src/main/java/org/apache/iceberg/view/ViewOperations.java create mode 100644 core/src/main/java/org/apache/iceberg/view/ViewUtil.java create mode 100644 core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java create mode 100644 core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryViewCatalog.java create mode 100644 core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java diff --git a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java index 3956e9192aaf..ee7666e9a7e9 100644 --- a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java +++ b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java @@ -28,7 +28,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; -import org.apache.iceberg.BaseMetastoreCatalog; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; @@ -42,23 +41,30 @@ import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.view.BaseMetastoreViewCatalog; +import org.apache.iceberg.view.BaseViewOperations; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewUtil; /** * Catalog implementation that uses in-memory data-structures to store the namespaces and tables. * This class doesn't touch external resources and can be utilized to write unit tests without side * effects. It uses {@link InMemoryFileIO}. */ -public class InMemoryCatalog extends BaseMetastoreCatalog implements SupportsNamespaces, Closeable { +public class InMemoryCatalog extends BaseMetastoreViewCatalog + implements SupportsNamespaces, Closeable { private static final Joiner SLASH = Joiner.on("/"); private static final Joiner DOT = Joiner.on("."); private final ConcurrentMap> namespaces; private final ConcurrentMap tables; + private final ConcurrentMap views; private FileIO io; private String catalogName; private String warehouseLocation; @@ -66,6 +72,7 @@ public class InMemoryCatalog extends BaseMetastoreCatalog implements SupportsNam public InMemoryCatalog() { this.namespaces = Maps.newConcurrentMap(); this.tables = Maps.newConcurrentMap(); + this.views = Maps.newConcurrentMap(); } @Override @@ -278,15 +285,69 @@ public List listNamespaces(Namespace namespace) throws NoSuchNamespac public void close() throws IOException { namespaces.clear(); tables.clear(); + views.clear(); + } + + @Override + public List listViews(Namespace namespace) { + if (!namespaceExists(namespace) && !namespace.isEmpty()) { + throw new NoSuchNamespaceException( + "Cannot list views for namespace. Namespace does not exist: %s", namespace); + } + + return views.keySet().stream() + .filter(v -> namespace.isEmpty() || v.namespace().equals(namespace)) + .sorted(Comparator.comparing(TableIdentifier::toString)) + .collect(Collectors.toList()); + } + + @Override + protected InMemoryViewOperations newViewOps(TableIdentifier identifier) { + return new InMemoryViewOperations(io, identifier); + } + + @Override + public boolean dropView(TableIdentifier identifier) { + return null != views.remove(identifier); + } + + @Override + public synchronized void renameView(TableIdentifier from, TableIdentifier to) { + if (from.equals(to)) { + return; + } + + if (!namespaceExists(to.namespace())) { + throw new NoSuchNamespaceException( + "Cannot rename %s to %s. Namespace does not exist: %s", from, to, to.namespace()); + } + + String fromViewLocation = views.get(from); + if (null == fromViewLocation) { + throw new NoSuchViewException("Cannot rename %s to %s. View does not exist", from, to); + } + + if (tables.containsKey(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. Table already exists", from, to); + } + + if (views.containsKey(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. View already exists", from, to); + } + + views.put(to, fromViewLocation); + views.remove(from); } private class InMemoryTableOperations extends BaseMetastoreTableOperations { private final FileIO fileIO; private final TableIdentifier tableIdentifier; + private final String fullTableName; InMemoryTableOperations(FileIO fileIO, TableIdentifier tableIdentifier) { this.fileIO = fileIO; this.tableIdentifier = tableIdentifier; + this.fullTableName = fullTableName(catalogName, tableIdentifier); } @Override @@ -300,8 +361,8 @@ public void doRefresh() { } @Override - public void doCommit(TableMetadata base, TableMetadata metadata) { - String newLocation = writeNewMetadata(metadata, currentVersion() + 1); + public synchronized void doCommit(TableMetadata base, TableMetadata metadata) { + String newLocation = writeNewMetadataIfRequired(base == null, metadata); String oldLocation = base == null ? null : base.metadataFileLocation(); if (null == base && !namespaceExists(tableIdentifier.namespace())) { @@ -310,6 +371,10 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { tableIdentifier, tableIdentifier.namespace()); } + if (views.containsKey(tableIdentifier)) { + throw new AlreadyExistsException("View with same name already exists: %s", tableIdentifier); + } + tables.compute( tableIdentifier, (k, existingLocation) -> { @@ -334,7 +399,72 @@ public FileIO io() { @Override protected String tableName() { - return tableIdentifier.toString(); + return fullTableName; + } + } + + private class InMemoryViewOperations extends BaseViewOperations { + private final FileIO io; + private final TableIdentifier identifier; + private final String fullViewName; + + InMemoryViewOperations(FileIO io, TableIdentifier identifier) { + this.io = io; + this.identifier = identifier; + this.fullViewName = ViewUtil.fullViewName(catalogName, identifier); + } + + @Override + public void doRefresh() { + String latestLocation = views.get(identifier); + if (latestLocation == null) { + disableRefresh(); + } else { + refreshFromMetadataLocation(latestLocation); + } + } + + @Override + public synchronized void doCommit(ViewMetadata base, ViewMetadata metadata) { + String newLocation = writeNewMetadataIfRequired(metadata); + String oldLocation = base == null ? null : currentMetadataLocation(); + + if (null == base && !namespaceExists(identifier.namespace())) { + throw new NoSuchNamespaceException( + "Cannot create view %s. Namespace does not exist: %s", + identifier, identifier.namespace()); + } + + if (tables.containsKey(identifier)) { + throw new AlreadyExistsException("Table with same name already exists: %s", identifier); + } + + views.compute( + identifier, + (k, existingLocation) -> { + if (!Objects.equal(existingLocation, oldLocation)) { + if (null == base) { + throw new AlreadyExistsException("View already exists: %s", identifier); + } + + throw new CommitFailedException( + "Cannot commit to view %s metadata location from %s to %s " + + "because it has been concurrently modified to %s", + identifier, oldLocation, newLocation, existingLocation); + } + + return newLocation; + }); + } + + @Override + public FileIO io() { + return io; + } + + @Override + protected String viewName() { + return fullViewName; } } } diff --git a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java new file mode 100644 index 000000000000..4d1309a97424 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java @@ -0,0 +1,190 @@ +/* + * 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.view; + +import java.util.Map; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.catalog.ViewCatalog; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchViewException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public abstract class BaseMetastoreViewCatalog extends BaseMetastoreCatalog implements ViewCatalog { + protected abstract ViewOperations newViewOps(TableIdentifier identifier); + + @Override + public void initialize(String name, Map properties) { + super.initialize(name, properties); + } + + @Override + public String name() { + return super.name(); + } + + @Override + public View loadView(TableIdentifier identifier) { + if (isValidIdentifier(identifier)) { + ViewOperations ops = newViewOps(identifier); + if (ops.current() == null) { + throw new NoSuchViewException("View does not exist: %s", identifier); + } else { + return new BaseView(newViewOps(identifier), ViewUtil.fullViewName(name(), identifier)); + } + } + + throw new NoSuchViewException("Invalid view identifier: %s", identifier); + } + + @Override + public ViewBuilder buildView(TableIdentifier identifier) { + return new BaseViewBuilder(identifier); + } + + protected class BaseViewBuilder implements ViewBuilder { + private final TableIdentifier identifier; + private final ImmutableViewVersion.Builder viewVersionBuilder = ImmutableViewVersion.builder(); + private final Map properties = Maps.newHashMap(); + private Schema schema; + + protected BaseViewBuilder(TableIdentifier identifier) { + Preconditions.checkArgument( + isValidIdentifier(identifier), "Invalid view identifier: %s", identifier); + this.identifier = identifier; + } + + @Override + public ViewBuilder withSchema(Schema newSchema) { + this.schema = newSchema; + viewVersionBuilder.schemaId(newSchema.schemaId()); + return this; + } + + @Override + public ViewBuilder withQuery(String dialect, String sql) { + viewVersionBuilder.addRepresentations( + ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build()); + return this; + } + + @Override + public ViewBuilder withDefaultCatalog(String defaultCatalog) { + viewVersionBuilder.defaultCatalog(defaultCatalog); + return this; + } + + @Override + public ViewBuilder withDefaultNamespace(Namespace namespace) { + viewVersionBuilder.defaultNamespace(namespace); + return this; + } + + @Override + public ViewBuilder withProperties(Map newProperties) { + this.properties.putAll(newProperties); + return this; + } + + @Override + public ViewBuilder withProperty(String key, String value) { + this.properties.put(key, value); + return this; + } + + @Override + public View create() { + ViewOperations ops = newViewOps(identifier); + if (null != ops.current()) { + throw new AlreadyExistsException("View already exists: %s", identifier); + } + + ViewVersion viewVersion = + viewVersionBuilder + .versionId(1) + .timestampMillis(System.currentTimeMillis()) + .putSummary("operation", "create") + .build(); + + ViewMetadata viewMetadata = + ViewMetadata.builder() + .setProperties(properties) + .setLocation(defaultWarehouseLocation(identifier)) + .setCurrentVersion(viewVersion, schema) + .build(); + + try { + ops.commit(null, viewMetadata); + } catch (CommitFailedException ignored) { + throw new AlreadyExistsException("View was created concurrently: %s", identifier); + } + + return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); + } + + @Override + public View replace() { + ViewOperations ops = newViewOps(identifier); + if (null == ops.current()) { + throw new NoSuchViewException("View does not exist: %s", identifier); + } + + ViewMetadata metadata = ops.current(); + int maxVersionId = + metadata.versions().stream() + .map(ViewVersion::versionId) + .max(Integer::compareTo) + .orElseGet(metadata::currentVersionId); + + ViewVersion viewVersion = + viewVersionBuilder + .versionId(maxVersionId + 1) + .timestampMillis(System.currentTimeMillis()) + .putSummary("operation", "replace") + .build(); + + ViewMetadata replacement = + ViewMetadata.buildFrom(metadata) + .setProperties(properties) + .setCurrentVersion(viewVersion, schema) + .build(); + + try { + ops.commit(metadata, replacement); + } catch (CommitFailedException ignored) { + throw new AlreadyExistsException("View was updated concurrently: %s", identifier); + } + + return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); + } + + @Override + public View createOrReplace() { + if (null == newViewOps(identifier).current()) { + return create(); + } else { + return replace(); + } + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/view/BaseView.java b/core/src/main/java/org/apache/iceberg/view/BaseView.java new file mode 100644 index 000000000000..a21bc2381f90 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/BaseView.java @@ -0,0 +1,89 @@ +/* + * 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.view; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; + +public class BaseView implements View, Serializable { + + private final ViewOperations ops; + private final String name; + + public BaseView(ViewOperations ops, String name) { + this.ops = ops; + this.name = name; + } + + @Override + public String name() { + return name; + } + + public ViewOperations operations() { + return ops; + } + + @Override + public Schema schema() { + return operations().current().schema(); + } + + @Override + public Map schemas() { + return operations().current().schemasById(); + } + + @Override + public ViewVersion currentVersion() { + return operations().current().currentVersion(); + } + + @Override + public Iterable versions() { + return operations().current().versions(); + } + + @Override + public ViewVersion version(int versionId) { + return operations().current().version(versionId); + } + + @Override + public List history() { + return operations().current().history(); + } + + @Override + public Map properties() { + return operations().current().properties(); + } + + @Override + public UpdateViewProperties updateProperties() { + return new PropertiesUpdate(ops); + } + + @Override + public ReplaceViewVersion replaceVersion() { + return new ViewVersionReplace(ops); + } +} diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java new file mode 100644 index 000000000000..f7270b9a35ed --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java @@ -0,0 +1,220 @@ +/* + * 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.view; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.Predicate; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchViewException; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.util.LocationUtil; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class BaseViewOperations implements ViewOperations { + private static final Logger LOG = LoggerFactory.getLogger(BaseViewOperations.class); + + private static final String METADATA_FOLDER_NAME = "metadata"; + + private ViewMetadata currentMetadata = null; + private String currentMetadataLocation = null; + private boolean shouldRefresh = true; + private int version = -1; + + protected BaseViewOperations() {} + + protected void requestRefresh() { + this.shouldRefresh = true; + } + + protected void disableRefresh() { + this.shouldRefresh = false; + } + + protected abstract void doRefresh(); + + protected abstract void doCommit(ViewMetadata base, ViewMetadata metadata); + + protected abstract String viewName(); + + protected abstract FileIO io(); + + protected String currentMetadataLocation() { + return currentMetadataLocation; + } + + protected int currentVersion() { + return version; + } + + @Override + public ViewMetadata current() { + if (shouldRefresh) { + return refresh(); + } + + return currentMetadata; + } + + @Override + public ViewMetadata refresh() { + boolean currentMetadataWasAvailable = currentMetadata != null; + try { + doRefresh(); + } catch (NoSuchViewException e) { + if (currentMetadataWasAvailable) { + LOG.warn("Could not find the view during refresh, setting current metadata to null", e); + shouldRefresh = true; + } + + currentMetadata = null; + currentMetadataLocation = null; + version = -1; + throw e; + } + + return current(); + } + + @Override + public void commit(ViewMetadata base, ViewMetadata metadata) { + // if the metadata is already out of date, reject it + if (base != current()) { + if (base != null) { + throw new CommitFailedException("Cannot commit: stale view metadata"); + } else { + // when current is non-null, the view exists. but when base is null, the commit is trying + // to create the view + throw new AlreadyExistsException("View already exists: %s", viewName()); + } + } + + // if the metadata is not changed, return early + if (base == metadata) { + LOG.info("Nothing to commit."); + return; + } + + long start = System.currentTimeMillis(); + doCommit(base, metadata); + requestRefresh(); + + LOG.info( + "Successfully committed to view {} in {} ms", + viewName(), + System.currentTimeMillis() - start); + } + + private String writeNewMetadata(ViewMetadata metadata, int newVersion) { + String newMetadataFilePath = newMetadataFilePath(metadata, newVersion); + OutputFile newMetadataLocation = io().newOutputFile(newMetadataFilePath); + + // write the new metadata + // use overwrite to avoid negative caching in S3. this is safe because the metadata location is + // always unique because it includes a UUID. + ViewMetadataParser.overwrite(metadata, newMetadataLocation); + + return newMetadataLocation.location(); + } + + protected String writeNewMetadataIfRequired(ViewMetadata metadata) { + return null != metadata.metadataFileLocation() + ? metadata.metadataFileLocation() + : writeNewMetadata(metadata, version + 1); + } + + private String newMetadataFilePath(ViewMetadata metadata, int newVersion) { + return metadataFileLocation( + metadata, String.format("%05d-%s%s", newVersion, UUID.randomUUID(), ".metadata.json")); + } + + private String metadataFileLocation(ViewMetadata metadata, String filename) { + return String.format( + "%s/%s/%s", + LocationUtil.stripTrailingSlash(metadata.location()), METADATA_FOLDER_NAME, filename); + } + + protected void refreshFromMetadataLocation(String newLocation) { + refreshFromMetadataLocation(newLocation, null, 20); + } + + protected void refreshFromMetadataLocation( + String newLocation, Predicate shouldRetry, int numRetries) { + refreshFromMetadataLocation( + newLocation, + shouldRetry, + numRetries, + metadataLocation -> ViewMetadataParser.read(io().newInputFile(metadataLocation))); + } + + protected void refreshFromMetadataLocation( + String newLocation, + Predicate shouldRetry, + int numRetries, + Function metadataLoader) { + if (!Objects.equal(currentMetadataLocation, newLocation)) { + LOG.info("Refreshing view metadata from new version: {}", newLocation); + + AtomicReference newMetadata = new AtomicReference<>(); + Tasks.foreach(newLocation) + .retry(numRetries) + .exponentialBackoff(100, 5000, 600000, 4.0 /* 100, 400, 1600, ... */) + .throwFailureWhenFinished() + .stopRetryOn(NotFoundException.class) // overridden if shouldRetry is non-null + .shouldRetryTest(shouldRetry) + .run(metadataLocation -> newMetadata.set(metadataLoader.apply(metadataLocation))); + + this.currentMetadata = newMetadata.get(); + this.currentMetadataLocation = newLocation; + this.version = parseVersion(newLocation); + } + + this.shouldRefresh = false; + } + + /** + * Parse the version from view metadata file name. + * + * @param metadataLocation view metadata file location + * @return version of the view metadata file in success case and -1 if the version is not parsable + * (as a sign that the metadata is not part of this catalog) + */ + private static int parseVersion(String metadataLocation) { + int versionStart = metadataLocation.lastIndexOf('/') + 1; // if '/' isn't found, this will be 0 + int versionEnd = metadataLocation.indexOf('-', versionStart); + if (versionEnd < 0) { + // found filesystem view's metadata + return -1; + } + + try { + return Integer.valueOf(metadataLocation.substring(versionStart, versionEnd)); + } catch (NumberFormatException e) { + LOG.warn("Unable to parse version from metadata location: {}", metadataLocation, e); + return -1; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/view/PropertiesUpdate.java b/core/src/main/java/org/apache/iceberg/view/PropertiesUpdate.java new file mode 100644 index 000000000000..8b2142c069ab --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/PropertiesUpdate.java @@ -0,0 +1,102 @@ +/* + * 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.view; + +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; + +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; + +class PropertiesUpdate implements UpdateViewProperties { + private final ViewOperations ops; + private final Map updates = Maps.newHashMap(); + private final Set removals = Sets.newHashSet(); + private ViewMetadata base; + + PropertiesUpdate(ViewOperations ops) { + this.ops = ops; + this.base = ops.current(); + } + + @Override + public Map apply() { + this.base = ops.refresh(); + + return internalApply(base).properties(); + } + + private ViewMetadata internalApply(ViewMetadata metadata) { + return ViewMetadata.buildFrom(metadata) + .setProperties(updates) + .removeProperties(removals) + .build(); + } + + @Override + public void commit() { + Tasks.foreach(ops) + .retry( + PropertyUtil.propertyAsInt( + base.properties(), COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) + .exponentialBackoff( + PropertyUtil.propertyAsInt( + base.properties(), COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), + PropertyUtil.propertyAsInt( + base.properties(), COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), + PropertyUtil.propertyAsInt( + base.properties(), COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), + 2.0 /* exponential */) + .onlyRetryOn(CommitFailedException.class) + .run(taskOps -> taskOps.commit(base, internalApply(base))); + } + + @Override + public UpdateViewProperties set(String key, String value) { + Preconditions.checkArgument(null != key, "Invalid key: null"); + Preconditions.checkArgument(null != value, "Invalid value: null"); + Preconditions.checkArgument( + !removals.contains(key), "Cannot remove and update the same key: %s", key); + + updates.put(key, value); + return this; + } + + @Override + public UpdateViewProperties remove(String key) { + Preconditions.checkArgument(null != key, "Invalid key: null"); + Preconditions.checkArgument( + !updates.containsKey(key), "Cannot remove and update the same key: %s", key); + + removals.add(key); + return this; + } +} diff --git a/core/src/main/java/org/apache/iceberg/view/ViewOperations.java b/core/src/main/java/org/apache/iceberg/view/ViewOperations.java new file mode 100644 index 000000000000..f9b3a9436f7f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/ViewOperations.java @@ -0,0 +1,59 @@ +/* + * 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.view; + +/** SPI interface to abstract view metadata access and updates. */ +public interface ViewOperations { + + /** + * Return the currently loaded view metadata, without checking for updates. + * + * @return view metadata + */ + ViewMetadata current(); + + /** + * Return the current view metadata after checking for updates. + * + * @return view metadata + */ + ViewMetadata refresh(); + + /** + * Replace the base view metadata with a new version. + * + *

This method should implement and document atomicity guarantees. + * + *

Implementations must check that the base metadata is current to avoid overwriting updates. + * Once the atomic commit operation succeeds, implementations must not perform any operations that + * may fail because failure in this method cannot be distinguished from commit failure. + * + *

Implementations must throw a {@link + * org.apache.iceberg.exceptions.CommitStateUnknownException} in cases where it cannot be + * determined if the commit succeeded or failed. For example if a network partition causes the + * confirmation of the commit to be lost, the implementation should throw a + * CommitStateUnknownException. This is important because downstream users of this API need to + * know whether they can clean up the commit or not, if the state is unknown then it is not safe + * to remove any files. All other exceptions will be treated as if the commit has failed. + * + * @param base view metadata on which changes were based + * @param metadata new view metadata with updates + */ + void commit(ViewMetadata base, ViewMetadata metadata); +} diff --git a/core/src/main/java/org/apache/iceberg/view/ViewUtil.java b/core/src/main/java/org/apache/iceberg/view/ViewUtil.java new file mode 100644 index 000000000000..b79e2c3ce3db --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/ViewUtil.java @@ -0,0 +1,29 @@ +/* + * 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.view; + +import org.apache.iceberg.catalog.TableIdentifier; + +public class ViewUtil { + private ViewUtil() {} + + public static String fullViewName(String catalog, TableIdentifier ident) { + return catalog + "." + ident; + } +} diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java new file mode 100644 index 000000000000..d6bd655da1b2 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java @@ -0,0 +1,132 @@ +/* + * 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.view; + +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; + +class ViewVersionReplace implements ReplaceViewVersion { + private final ViewOperations ops; + private final List viewRepresentationsToAdd = Lists.newArrayList(); + private ViewMetadata base; + private Namespace defaultNamespace; + private String defaultCatalog; + private Schema schema; + + ViewVersionReplace(ViewOperations ops) { + this.ops = ops; + this.base = ops.current(); + } + + @Override + public ViewVersion apply() { + Preconditions.checkState( + !viewRepresentationsToAdd.isEmpty(), "Cannot replace view without specifying a query"); + Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); + + this.base = ops.refresh(); + + ViewVersion viewVersion = base.currentVersion(); + int maxVersionId = + base.versions().stream() + .map(ViewVersion::versionId) + .max(Integer::compareTo) + .orElseGet(viewVersion::versionId); + + return ImmutableViewVersion.builder() + .versionId(maxVersionId + 1) + .timestampMillis(System.currentTimeMillis()) + .schemaId(schema.schemaId()) + .defaultNamespace(defaultNamespace) + .defaultCatalog(defaultCatalog) + .putSummary("operation", "replace") + .addAllRepresentations(viewRepresentationsToAdd) + .build(); + } + + @Override + public void commit() { + Tasks.foreach(ops) + .retry( + PropertyUtil.propertyAsInt( + base.properties(), COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) + .exponentialBackoff( + PropertyUtil.propertyAsInt( + base.properties(), COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), + PropertyUtil.propertyAsInt( + base.properties(), COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), + PropertyUtil.propertyAsInt( + base.properties(), COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), + 2.0 /* exponential */) + .onlyRetryOn(CommitFailedException.class) + .run( + taskOps -> { + ViewVersion newVersion = apply(); + // nothing to do if the version didn't change + if (this.base.currentVersion().equals(newVersion)) { + return; + } + + ViewMetadata updated = + ViewMetadata.buildFrom(this.base).setCurrentVersion(newVersion, schema).build(); + + taskOps.commit(base, updated); + }); + } + + @Override + public ReplaceViewVersion withQuery(String dialect, String sql) { + viewRepresentationsToAdd.add( + ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build()); + return this; + } + + @Override + public ReplaceViewVersion withSchema(Schema newSchema) { + this.schema = newSchema; + return this; + } + + @Override + public ReplaceViewVersion withDefaultCatalog(String catalog) { + this.defaultCatalog = catalog; + return this; + } + + @Override + public ReplaceViewVersion withDefaultNamespace(Namespace namespace) { + this.defaultNamespace = namespace; + return this; + } +} diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryViewCatalog.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryViewCatalog.java new file mode 100644 index 000000000000..76731f58a6be --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryViewCatalog.java @@ -0,0 +1,49 @@ +/* + * 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.inmemory; + +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.view.ViewCatalogTests; +import org.junit.jupiter.api.BeforeEach; + +public class TestInMemoryViewCatalog extends ViewCatalogTests { + private InMemoryCatalog catalog; + + @BeforeEach + public void before() { + this.catalog = new InMemoryCatalog(); + this.catalog.initialize("in-memory-catalog", ImmutableMap.of()); + } + + @Override + protected InMemoryCatalog catalog() { + return catalog; + } + + @Override + protected Catalog tableCatalog() { + return catalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return true; + } +} diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java new file mode 100644 index 000000000000..e4629dc020c7 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -0,0 +1,943 @@ +/* + * 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.view; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import org.apache.iceberg.Schema; +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.catalog.ViewCatalog; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchViewException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assumptions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public abstract class ViewCatalogTests { + protected static final Schema SCHEMA = + new Schema( + 1, + required(3, "id", Types.IntegerType.get(), "unique ID"), + required(4, "data", Types.StringType.get())); + + private static final Schema OTHER_SCHEMA = + new Schema(2, required(1, "some_id", Types.IntegerType.get())); + + protected abstract C catalog(); + + protected abstract Catalog tableCatalog(); + + protected boolean requiresNamespaceCreate() { + return false; + } + + @Test + public void basicCreateView() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(view).isNotNull(); + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + + // validate view settings + assertThat(view.name()).isEqualTo(catalog().name() + "." + identifier); + assertThat(view.properties()).isEmpty(); + assertThat(view.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(1); + assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); + assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); + + assertThat(view.currentVersion()) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(view.currentVersion().timestampMillis()) + .versionId(1) + .schemaId(SCHEMA.schemaId()) + .putSummary("operation", "create") + .defaultNamespace(identifier.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("spark") + .build()) + .build()); + + assertThat(catalog().dropView(identifier)).isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + } + + @Test + public void completeCreateView() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .withQuery("trino", "select * from ns.tbl using X") + .withProperty("prop1", "val1") + .withProperty("prop2", "val2") + .create(); + + assertThat(view).isNotNull(); + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + + // validate view settings + assertThat(view.name()).isEqualTo(catalog().name() + "." + identifier); + assertThat(view.properties()).isEqualTo(ImmutableMap.of("prop1", "val1", "prop2", "val2")); + assertThat(view.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(1); + assertThat(view.schema().schemaId()).isEqualTo(SCHEMA.schemaId()); + assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); + assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); + + assertThat(view.currentVersion()) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(view.currentVersion().timestampMillis()) + .versionId(1) + .schemaId(SCHEMA.schemaId()) + .putSummary("operation", "create") + .defaultNamespace(identifier.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("spark") + .build()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl using X") + .dialect("trino") + .build()) + .build()); + + assertThat(catalog().dropView(identifier)).isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + } + + @Test + public void createViewThatAlreadyExists() { + TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(viewIdentifier.namespace()); + } + + assertThat(catalog().viewExists(viewIdentifier)).isFalse(); + + View view = + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(view).isNotNull(); + assertThat(catalog().viewExists(viewIdentifier)).isTrue(); + + assertThatThrownBy( + () -> + catalog() + .buildView(viewIdentifier) + .withSchema(OTHER_SCHEMA) + .withQuery("spark", "select * from ns.tbl") + .withDefaultNamespace(viewIdentifier.namespace()) + .create()) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View already exists: ns.view"); + } + + @Test + public void createViewThatAlreadyExistsAsTable() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + TableIdentifier tableIdentifier = TableIdentifier.of("ns", "table"); + TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(viewIdentifier.namespace()); + } + + assertThat(catalog().viewExists(viewIdentifier)).isFalse(); + + View view = + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(view).isNotNull(); + assertThat(catalog().viewExists(tableIdentifier)).isFalse(); + assertThat(catalog().viewExists(viewIdentifier)).isTrue(); + + assertThatThrownBy( + () -> + catalog() + .buildView(viewIdentifier) + .withSchema(OTHER_SCHEMA) + .withQuery("spark", "select * from ns.tbl") + .withDefaultNamespace(viewIdentifier.namespace()) + .create()) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View already exists: ns.view"); + + tableCatalog().buildTable(tableIdentifier, SCHEMA).create(); + assertThat(tableCatalog().tableExists(tableIdentifier)).isTrue(); + assertThat(catalog().viewExists(tableIdentifier)).isFalse(); + + assertThatThrownBy( + () -> + catalog() + .buildView(tableIdentifier) + .withSchema(OTHER_SCHEMA) + .withDefaultNamespace(tableIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create()) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("Table with same name already exists: ns.table"); + } + + @Test + public void createTableThatAlreadyExistsAsView() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + TableIdentifier viewOne = TableIdentifier.of("ns", "viewOne"); + TableIdentifier viewTwo = TableIdentifier.of("ns", "viewTwo"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(viewOne.namespace()); + } + + assertThat(catalog().viewExists(viewOne)).isFalse(); + assertThat(tableCatalog().tableExists(viewTwo)).isFalse(); + + for (TableIdentifier identifier : Arrays.asList(viewTwo, viewOne)) { + assertThat( + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create()) + .isNotNull(); + + assertThat(catalog().viewExists(identifier)).isTrue(); + assertThat(tableCatalog().tableExists(identifier)).isFalse(); + } + + assertThatThrownBy(() -> tableCatalog().buildTable(viewTwo, SCHEMA).create()) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View with same name already exists: ns.viewTwo"); + } + + @Test + public void renameView() { + TableIdentifier from = TableIdentifier.of("ns", "view"); + TableIdentifier to = TableIdentifier.of("ns", "renamedView"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(from.namespace()); + } + + catalog() + .buildView(from) + .withSchema(SCHEMA) + .withDefaultNamespace(from.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(from)).as("View should exist").isTrue(); + assertThat(catalog().listViews(from.namespace())).containsExactly(from); + + catalog().renameView(from, to); + + assertThat(catalog().listViews(to.namespace())).containsExactly(to); + assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); + assertThat(catalog().viewExists(to)).as("View should exist").isTrue(); + + View view = catalog().loadView(to); + assertThat(view).isNotNull(); + + // validate view settings + assertThat(view.name()).isEqualTo(catalog().name() + "." + to); + assertThat(view.properties()).isEmpty(); + assertThat(view.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(1); + assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); + assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); + + assertThat(view.currentVersion()) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(view.currentVersion().timestampMillis()) + .versionId(1) + .schemaId(SCHEMA.schemaId()) + .putSummary("operation", "create") + .defaultNamespace(to.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("spark") + .build()) + .build()); + + assertThat(catalog().dropView(to)).isTrue(); + assertThat(catalog().viewExists(to)).as("View should not exist").isFalse(); + } + + @Test + public void renameViewUsingDifferentNamespace() { + TableIdentifier from = TableIdentifier.of("ns", "view"); + TableIdentifier to = TableIdentifier.of("other_ns", "renamedView"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(from.namespace()); + catalog().createNamespace(to.namespace()); + } + + catalog() + .buildView(from) + .withSchema(SCHEMA) + .withDefaultNamespace(from.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().listViews(from.namespace())).containsExactly(from); + assertThat(catalog().viewExists(from)).as("View should exist").isTrue(); + + catalog().renameView(from, to); + + assertThat(catalog().listViews(to.namespace())).containsExactly(to); + assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); + assertThat(catalog().viewExists(to)).as("View should exist").isTrue(); + + View view = catalog().loadView(to); + assertThat(view).isNotNull(); + + // validate view settings + assertThat(view.name()).isEqualTo(catalog().name() + "." + to); + assertThat(view.properties()).isEmpty(); + assertThat(view.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(1); + assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); + assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); + + assertThat(view.currentVersion()) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(view.currentVersion().timestampMillis()) + .versionId(1) + .schemaId(SCHEMA.schemaId()) + .putSummary("operation", "create") + .defaultNamespace(from.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("spark") + .build()) + .build()); + + assertThat(catalog().dropView(from)).isFalse(); + assertThat(catalog().dropView(to)).isTrue(); + assertThat(catalog().viewExists(to)).as("View should not exist").isFalse(); + } + + @Test + public void renameViewNamespaceMissing() { + TableIdentifier from = TableIdentifier.of("ns", "view"); + TableIdentifier to = TableIdentifier.of("non_existing", "renamedView"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(from.namespace()); + } + + catalog() + .buildView(from) + .withSchema(SCHEMA) + .withDefaultNamespace(from.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(from)).as("View should exist").isTrue(); + + assertThatThrownBy(() -> catalog().renameView(from, to)) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: non_existing"); + } + + @Test + public void renameViewSourceMissing() { + TableIdentifier from = TableIdentifier.of("ns", "view"); + TableIdentifier to = TableIdentifier.of("ns", "renamedView"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(from.namespace()); + } + + assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); + + assertThatThrownBy(() -> catalog().renameView(from, to)) + .isInstanceOf(NoSuchViewException.class) + .hasMessageContaining("View does not exist"); + + assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); + assertThat(catalog().viewExists(to)).as("View should not exist").isFalse(); + } + + @Test + public void renameViewTargetAlreadyExists() { + TableIdentifier from = TableIdentifier.of("ns", "view"); + TableIdentifier to = TableIdentifier.of("ns", "renamedView"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(from.namespace()); + } + + for (TableIdentifier viewIdentifier : ImmutableList.of(from, to)) { + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(from.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + } + + assertThatThrownBy(() -> catalog().renameView(from, to)) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Cannot rename ns.view to ns.renamedView. View already exists"); + + // rename view where a table with the same name already exists + TableIdentifier identifier = TableIdentifier.of("ns", "tbl"); + tableCatalog().buildTable(identifier, SCHEMA).create(); + + assertThatThrownBy(() -> catalog().renameView(from, identifier)) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Cannot rename ns.view to ns.tbl. Table already exists"); + } + + @Test + public void listViews() { + Namespace ns1 = Namespace.of("ns1"); + Namespace ns2 = Namespace.of("ns2"); + + TableIdentifier tableIdentifier = TableIdentifier.of(ns1, "table"); + TableIdentifier view1 = TableIdentifier.of(ns1, "view1"); + TableIdentifier view2 = TableIdentifier.of(ns2, "view2"); + TableIdentifier view3 = TableIdentifier.of(ns2, "view3"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(ns1); + catalog().createNamespace(ns2); + } + + if (null != tableCatalog()) { + tableCatalog().buildTable(tableIdentifier, SCHEMA).create(); + assertThat(tableCatalog().listTables(ns1)).containsExactly(tableIdentifier); + assertThat(tableCatalog().listTables(ns2)).isEmpty(); + } + + assertThat(catalog().listViews(ns1)).isEmpty(); + assertThat(catalog().listViews(ns2)).isEmpty(); + + catalog() + .buildView(view1) + .withSchema(SCHEMA) + .withDefaultNamespace(view1.namespace()) + .withQuery("spark", "select * from ns1.tbl") + .create(); + + assertThat(catalog().listViews(ns1)).containsExactly(view1); + assertThat(catalog().listViews(ns2)).isEmpty(); + + catalog() + .buildView(view2) + .withSchema(SCHEMA) + .withDefaultNamespace(view2.namespace()) + .withQuery("spark", "select * from ns1.tbl") + .create(); + + assertThat(catalog().listViews(ns1)).containsExactly(view1); + assertThat(catalog().listViews(ns2)).containsExactly(view2); + + catalog() + .buildView(view3) + .withSchema(SCHEMA) + .withDefaultNamespace(view3.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().listViews(ns1)).containsExactly(view1); + assertThat(catalog().listViews(ns2)).containsExactlyInAnyOrder(view2, view3); + + if (null != tableCatalog()) { + assertThat(tableCatalog().listTables(ns1)).containsExactly(tableIdentifier); + assertThat(tableCatalog().listTables(ns2)).isEmpty(); + } + + assertThat(catalog().dropView(view2)).isTrue(); + assertThat(catalog().listViews(ns1)).containsExactly(view1); + assertThat(catalog().listViews(ns2)).containsExactly(view3); + + assertThat(catalog().dropView(view3)).isTrue(); + assertThat(catalog().listViews(ns1)).containsExactly(view1); + assertThat(catalog().listViews(ns2)).isEmpty(); + + assertThat(catalog().dropView(view1)).isTrue(); + assertThat(catalog().listViews(ns1)).isEmpty(); + assertThat(catalog().listViews(ns2)).isEmpty(); + } + + @ParameterizedTest(name = ".createOrReplace() = {arguments}") + @ValueSource(booleans = {false, true}) + public void createOrReplaceView(boolean useCreateOrReplace) { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + ViewBuilder viewBuilder = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .withProperty("prop1", "val1") + .withProperty("prop2", "val2"); + View view = useCreateOrReplace ? viewBuilder.createOrReplace() : viewBuilder.create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + + // validate view settings + assertThat(view.name()).isEqualTo(catalog().name() + "." + identifier); + assertThat(view.properties()).isEqualTo(ImmutableMap.of("prop1", "val1", "prop2", "val2")); + assertThat(view.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(1); + assertThat(view.schema().schemaId()).isEqualTo(SCHEMA.schemaId()); + assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); + assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); + + ViewVersion viewVersion = view.currentVersion(); + assertThat(viewVersion).isNotNull(); + assertThat(viewVersion.versionId()).isEqualTo(1); + assertThat(viewVersion.schemaId()).isEqualTo(SCHEMA.schemaId()); + assertThat(viewVersion.summary()).hasSize(1).containsEntry("operation", "create"); + assertThat(viewVersion.operation()).isEqualTo("create"); + assertThat(viewVersion.defaultNamespace()).isEqualTo(identifier.namespace()); + assertThat(viewVersion.representations()) + .hasSize(1) + .containsExactly( + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("spark") + .build()); + + viewBuilder = + catalog() + .buildView(identifier) + .withSchema(OTHER_SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("trino", "select count(*) from ns.tbl") + .withProperty("replacedProp1", "val1") + .withProperty("replacedProp2", "val2"); + View replacedView = useCreateOrReplace ? viewBuilder.createOrReplace() : viewBuilder.replace(); + + // validate replaced view settings + assertThat(replacedView.name()).isEqualTo(catalog().name() + "." + identifier); + assertThat(replacedView.properties()) + .hasSize(4) + .containsEntry("prop1", "val1") + .containsEntry("prop2", "val2") + .containsEntry("replacedProp1", "val1") + .containsEntry("replacedProp2", "val2"); + assertThat(replacedView.history()) + .hasSize(2) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(1); + assertThat(replacedView.history()) + .element(1) + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(2); + + assertThat(replacedView.schema().schemaId()).isEqualTo(OTHER_SCHEMA.schemaId()); + assertThat(replacedView.schema().asStruct()).isEqualTo(OTHER_SCHEMA.asStruct()); + assertThat(replacedView.schemas()) + .hasSize(2) + .containsKey(SCHEMA.schemaId()) + .containsKey(OTHER_SCHEMA.schemaId()); + + ViewVersion replacedViewVersion = replacedView.currentVersion(); + assertThat(replacedView.versions()) + .hasSize(2) + .containsExactly(viewVersion, replacedViewVersion); + assertThat(replacedViewVersion).isNotNull(); + assertThat(replacedViewVersion.versionId()).isEqualTo(2); + assertThat(replacedViewVersion.schemaId()).isEqualTo(OTHER_SCHEMA.schemaId()); + assertThat(replacedViewVersion.operation()).isEqualTo("replace"); + assertThat(replacedViewVersion.summary()).hasSize(1).containsEntry("operation", "replace"); + assertThat(replacedViewVersion.representations()) + .hasSize(1) + .containsExactly( + ImmutableSQLViewRepresentation.builder() + .sql("select count(*) from ns.tbl") + .dialect("trino") + .build()); + + assertThat(catalog().dropView(identifier)).isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + } + + @Test + public void updateViewProperties() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(view.properties()).isEmpty(); + ViewVersion viewVersion = view.currentVersion(); + assertThat(viewVersion.operation()).isEqualTo("create"); + assertThat(viewVersion.versionId()).isEqualTo(1); + assertThat(view.history()).hasSize(1); + assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(view.versions()).hasSize(1).containsExactly(viewVersion); + + assertThatThrownBy( + () -> catalog().loadView(identifier).updateProperties().set(null, "new-val1").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid key: null"); + + assertThatThrownBy( + () -> catalog().loadView(identifier).updateProperties().set("key1", null).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid value: null"); + + assertThatThrownBy( + () -> catalog().loadView(identifier).updateProperties().remove(null).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid key: null"); + + assertThatThrownBy( + () -> + catalog() + .loadView(identifier) + .updateProperties() + .set("key1", "x") + .set("key3", "y") + .remove("key2") + .set("key2", "z") + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot remove and update the same key: key2"); + + view.updateProperties().set("key1", "val1").set("key2", "val2").remove("non-existing").commit(); + + View updatedView = catalog().loadView(identifier); + assertThat(updatedView.properties()) + .hasSize(2) + .containsEntry("key1", "val1") + .containsEntry("key2", "val2"); + assertThat(updatedView.history()).hasSize(1).isEqualTo(view.history()); + assertThat(updatedView.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(updatedView.versions()).hasSize(1).containsExactly(viewVersion); + + // updating properties doesn't change the view version + ViewVersion updatedViewVersion = updatedView.currentVersion(); + assertThat(updatedViewVersion).isNotNull(); + assertThat(updatedViewVersion.versionId()).isEqualTo(viewVersion.versionId()); + assertThat(updatedViewVersion.summary()).isEqualTo(viewVersion.summary()); + assertThat(updatedViewVersion.operation()).isEqualTo(viewVersion.operation()); + + assertThatThrownBy( + () -> + catalog() + .loadView(identifier) + .updateProperties() + .set("key1", "new-val1") + .set("key3", "val3") + .remove("key2") + .set("key2", "new-val2") + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot remove and update the same key: key2"); + + view.updateProperties().set("key1", "new-val1").set("key3", "val3").remove("key2").commit(); + + View updatedView2 = catalog().loadView(identifier); + assertThat(updatedView2.properties()) + .hasSize(2) + .containsEntry("key1", "new-val1") + .containsEntry("key3", "val3"); + assertThat(updatedView2.history()).hasSize(1).isEqualTo(view.history()); + assertThat(updatedView2.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(updatedView2.versions()).hasSize(1).containsExactly(viewVersion); + + ViewVersion updatedViewVersion2 = updatedView2.currentVersion(); + assertThat(updatedViewVersion2).isNotNull(); + assertThat(updatedViewVersion2.versionId()).isEqualTo(viewVersion.versionId()); + assertThat(updatedViewVersion2.summary()).isEqualTo(viewVersion.summary()); + assertThat(updatedViewVersion2.operation()).isEqualTo(viewVersion.operation()); + + assertThat(catalog().dropView(identifier)).isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + } + + @Test + public void replaceViewVersion() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + SQLViewRepresentation spark = + ImmutableSQLViewRepresentation.builder() + .dialect("spark") + .sql("select * from ns.tbl") + .build(); + + SQLViewRepresentation trino = + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("trino") + .build(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery(trino.dialect(), trino.sql()) + .withQuery(spark.dialect(), spark.sql()) + .create(); + + ViewVersion viewVersion = view.currentVersion(); + assertThat(view.properties()).isEmpty(); + assertThat(view.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(viewVersion.versionId()); + assertThat(view.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(view.currentVersion().versionId()); + assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(viewVersion.operation()).isEqualTo("create"); + assertThat(viewVersion.versionId()).isEqualTo(1); + assertThat(viewVersion.representations()).hasSize(2).containsExactly(trino, spark); + assertThat(view.versions()).hasSize(1).containsExactly(viewVersion); + + view.replaceVersion() + .withSchema(OTHER_SCHEMA) + .withQuery(trino.dialect(), trino.sql()) + .withDefaultCatalog("default") + .withDefaultNamespace(identifier.namespace()) + .commit(); + + View updatedView = catalog().loadView(identifier); + assertThat(updatedView.properties()).isEmpty(); + assertThat(updatedView.history()) + .hasSize(2) + .element(0) + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(viewVersion.versionId()); + assertThat(updatedView.history()) + .hasSize(2) + .element(1) + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(updatedView.currentVersion().versionId()); + assertThat(updatedView.schemas()) + .hasSize(2) + .containsKey(SCHEMA.schemaId()) + .containsKey(OTHER_SCHEMA.schemaId()); + assertThat(updatedView.versions()) + .hasSize(2) + .containsExactly(viewVersion, updatedView.currentVersion()); + + ViewVersion updatedViewVersion = updatedView.currentVersion(); + assertThat(updatedViewVersion).isNotNull(); + assertThat(updatedViewVersion.versionId()).isEqualTo(viewVersion.versionId() + 1); + assertThat(updatedViewVersion.summary()).hasSize(1).containsEntry("operation", "replace"); + assertThat(updatedViewVersion.operation()).isEqualTo("replace"); + assertThat(updatedViewVersion.representations()).hasSize(1).containsExactly(trino); + assertThat(updatedViewVersion.schemaId()).isEqualTo(OTHER_SCHEMA.schemaId()); + assertThat(updatedViewVersion.defaultCatalog()).isEqualTo("default"); + assertThat(updatedViewVersion.defaultNamespace()).isEqualTo(identifier.namespace()); + + SQLViewRepresentation updatedSpark = + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.updated_tbl") + .dialect("spark") + .build(); + + view.replaceVersion() + .withQuery(updatedSpark.dialect(), updatedSpark.sql()) + .withDefaultNamespace(identifier.namespace()) + .withSchema(OTHER_SCHEMA) + .commit(); + + View updatedView2 = catalog().loadView(identifier); + assertThat(updatedView2.properties()).isEmpty(); + assertThat(updatedView2.history()) + .hasSize(3) + .element(0) + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(viewVersion.versionId()); + assertThat(updatedView2.history()) + .element(1) + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(updatedViewVersion.versionId()); + assertThat(updatedView2.history()) + .element(2) + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(updatedView2.currentVersion().versionId()); + assertThat(updatedView.schemas()) + .hasSize(2) + .containsKey(SCHEMA.schemaId()) + .containsKey(OTHER_SCHEMA.schemaId()); + assertThat(updatedView2.versions()) + .hasSize(3) + .containsExactly(viewVersion, updatedViewVersion, updatedView2.currentVersion()); + + ViewVersion updatedViewVersion2 = updatedView2.currentVersion(); + assertThat(updatedViewVersion2).isNotNull(); + assertThat(updatedViewVersion2.versionId()).isEqualTo(updatedViewVersion.versionId() + 1); + assertThat(updatedViewVersion2.summary()).hasSize(1).containsEntry("operation", "replace"); + assertThat(updatedViewVersion2.operation()).isEqualTo("replace"); + assertThat(updatedViewVersion2.representations()).hasSize(1).containsExactly(updatedSpark); + + assertThat(catalog().dropView(identifier)).isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + } + + @Test + public void replaceViewVersionErrorCases() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + SQLViewRepresentation trino = + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("trino") + .build(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery(trino.dialect(), trino.sql()) + .create(); + + // empty commits are not allowed + assertThatThrownBy(() -> view.replaceVersion().commit()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot replace view without specifying a query"); + + // schema is required + assertThatThrownBy( + () -> + view.replaceVersion() + .withQuery(trino.dialect(), trino.sql()) + .withDefaultNamespace(identifier.namespace()) + .commit()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot replace view without specifying schema"); + } +} From 852cae5c2ce68ff3474f1d923a2d59c743b9f98d Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 22 Sep 2023 15:20:38 +0200 Subject: [PATCH 02/14] review feedback --- .../view/BaseMetastoreViewCatalog.java | 31 +- .../apache/iceberg/view/ViewOperations.java | 10 +- .../iceberg/view/ViewVersionReplace.java | 54 +- .../apache/iceberg/view/ViewCatalogTests.java | 486 ++++++++---------- 4 files changed, 274 insertions(+), 307 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java index 4d1309a97424..f6d0da45b623 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java @@ -114,7 +114,25 @@ public ViewBuilder withProperty(String key, String value) { @Override public View create() { + return create(newViewOps(identifier)); + } + + @Override + public View replace() { + return replace(newViewOps(identifier)); + } + + @Override + public View createOrReplace() { ViewOperations ops = newViewOps(identifier); + if (null == ops.current()) { + return create(ops); + } else { + return replace(ops); + } + } + + private View create(ViewOperations ops) { if (null != ops.current()) { throw new AlreadyExistsException("View already exists: %s", identifier); } @@ -142,9 +160,7 @@ public View create() { return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); } - @Override - public View replace() { - ViewOperations ops = newViewOps(identifier); + private View replace(ViewOperations ops) { if (null == ops.current()) { throw new NoSuchViewException("View does not exist: %s", identifier); } @@ -177,14 +193,5 @@ public View replace() { return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); } - - @Override - public View createOrReplace() { - if (null == newViewOps(identifier).current()) { - return create(); - } else { - return replace(); - } - } } } diff --git a/core/src/main/java/org/apache/iceberg/view/ViewOperations.java b/core/src/main/java/org/apache/iceberg/view/ViewOperations.java index f9b3a9436f7f..37133161b9cc 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewOperations.java @@ -44,13 +44,15 @@ public interface ViewOperations { * Once the atomic commit operation succeeds, implementations must not perform any operations that * may fail because failure in this method cannot be distinguished from commit failure. * - *

Implementations must throw a {@link + *

Implementations should throw a {@link * org.apache.iceberg.exceptions.CommitStateUnknownException} in cases where it cannot be * determined if the commit succeeded or failed. For example if a network partition causes the * confirmation of the commit to be lost, the implementation should throw a - * CommitStateUnknownException. This is important because downstream users of this API need to - * know whether they can clean up the commit or not, if the state is unknown then it is not safe - * to remove any files. All other exceptions will be treated as if the commit has failed. + * CommitStateUnknownException. An unknown state indicates to downstream users of this API that it + * is not safe to perform clean up and remove any files. In general, strict metadata cleanup will + * only trigger cleanups when the commit fails with an exception implementing the marker interface + * {@link org.apache.iceberg.exceptions.CleanableFailure}. All other exceptions will be treated as + * if the commit has failed. * * @param base view metadata on which changes were based * @param metadata new view metadata with updates diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java index d6bd655da1b2..387771f64b2d 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java @@ -38,7 +38,7 @@ class ViewVersionReplace implements ReplaceViewVersion { private final ViewOperations ops; - private final List viewRepresentationsToAdd = Lists.newArrayList(); + private final List representations = Lists.newArrayList(); private ViewMetadata base; private Namespace defaultNamespace; private String defaultCatalog; @@ -51,28 +51,35 @@ class ViewVersionReplace implements ReplaceViewVersion { @Override public ViewVersion apply() { + this.base = ops.refresh(); + + return internalApply(base).currentVersion(); + } + + private ViewMetadata internalApply(ViewMetadata metadata) { Preconditions.checkState( - !viewRepresentationsToAdd.isEmpty(), "Cannot replace view without specifying a query"); + !representations.isEmpty(), "Cannot replace view without specifying a query"); Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); - this.base = ops.refresh(); - - ViewVersion viewVersion = base.currentVersion(); + ViewVersion viewVersion = metadata.currentVersion(); int maxVersionId = - base.versions().stream() + metadata.versions().stream() .map(ViewVersion::versionId) .max(Integer::compareTo) .orElseGet(viewVersion::versionId); - return ImmutableViewVersion.builder() - .versionId(maxVersionId + 1) - .timestampMillis(System.currentTimeMillis()) - .schemaId(schema.schemaId()) - .defaultNamespace(defaultNamespace) - .defaultCatalog(defaultCatalog) - .putSummary("operation", "replace") - .addAllRepresentations(viewRepresentationsToAdd) - .build(); + ViewVersion newVersion = + ImmutableViewVersion.builder() + .versionId(maxVersionId + 1) + .timestampMillis(System.currentTimeMillis()) + .schemaId(schema.schemaId()) + .defaultNamespace(defaultNamespace) + .defaultCatalog(defaultCatalog) + .putSummary("operation", "replace") + .addAllRepresentations(representations) + .build(); + + return ViewMetadata.buildFrom(metadata).setCurrentVersion(newVersion, schema).build(); } @Override @@ -90,25 +97,12 @@ public void commit() { base.properties(), COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), 2.0 /* exponential */) .onlyRetryOn(CommitFailedException.class) - .run( - taskOps -> { - ViewVersion newVersion = apply(); - // nothing to do if the version didn't change - if (this.base.currentVersion().equals(newVersion)) { - return; - } - - ViewMetadata updated = - ViewMetadata.buildFrom(this.base).setCurrentVersion(newVersion, schema).build(); - - taskOps.commit(base, updated); - }); + .run(taskOps -> taskOps.commit(base, internalApply(base))); } @Override public ReplaceViewVersion withQuery(String dialect, String sql) { - viewRepresentationsToAdd.add( - ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build()); + representations.add(ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build()); return this; } diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index e4629dc020c7..019040f81dfa 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -22,7 +22,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.util.Arrays; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -33,7 +32,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assumptions; import org.junit.jupiter.api.Test; @@ -80,8 +78,7 @@ public void basicCreateView() { assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); // validate view settings - assertThat(view.name()).isEqualTo(catalog().name() + "." + identifier); - assertThat(view.properties()).isEmpty(); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalog().name(), identifier)); assertThat(view.history()) .hasSize(1) .first() @@ -135,8 +132,8 @@ public void completeCreateView() { assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); // validate view settings - assertThat(view.name()).isEqualTo(catalog().name() + "." + identifier); - assertThat(view.properties()).isEqualTo(ImmutableMap.of("prop1", "val1", "prop2", "val2")); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalog().name(), identifier)); + assertThat(view.properties()).containsEntry("prop1", "val1").containsEntry("prop2", "val2"); assertThat(view.history()) .hasSize(1) .first() @@ -173,32 +170,32 @@ public void completeCreateView() { @Test public void createViewThatAlreadyExists() { - TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); + TableIdentifier identifier = TableIdentifier.of("ns", "view"); if (requiresNamespaceCreate()) { - catalog().createNamespace(viewIdentifier.namespace()); + catalog().createNamespace(identifier.namespace()); } - assertThat(catalog().viewExists(viewIdentifier)).isFalse(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); View view = catalog() - .buildView(viewIdentifier) + .buildView(identifier) .withSchema(SCHEMA) - .withDefaultNamespace(viewIdentifier.namespace()) + .withDefaultNamespace(identifier.namespace()) .withQuery("spark", "select * from ns.tbl") .create(); assertThat(view).isNotNull(); - assertThat(catalog().viewExists(viewIdentifier)).isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); assertThatThrownBy( () -> catalog() - .buildView(viewIdentifier) + .buildView(identifier) .withSchema(OTHER_SCHEMA) .withQuery("spark", "select * from ns.tbl") - .withDefaultNamespace(viewIdentifier.namespace()) + .withDefaultNamespace(identifier.namespace()) .create()) .isInstanceOf(AlreadyExistsException.class) .hasMessageStartingWith("View already exists: ns.view"); @@ -211,40 +208,16 @@ public void createViewThatAlreadyExistsAsTable() { .isNotNull(); TableIdentifier tableIdentifier = TableIdentifier.of("ns", "table"); - TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); if (requiresNamespaceCreate()) { - catalog().createNamespace(viewIdentifier.namespace()); + catalog().createNamespace(tableIdentifier.namespace()); } - assertThat(catalog().viewExists(viewIdentifier)).isFalse(); - - View view = - catalog() - .buildView(viewIdentifier) - .withSchema(SCHEMA) - .withDefaultNamespace(viewIdentifier.namespace()) - .withQuery("spark", "select * from ns.tbl") - .create(); - - assertThat(view).isNotNull(); - assertThat(catalog().viewExists(tableIdentifier)).isFalse(); - assertThat(catalog().viewExists(viewIdentifier)).isTrue(); - - assertThatThrownBy( - () -> - catalog() - .buildView(viewIdentifier) - .withSchema(OTHER_SCHEMA) - .withQuery("spark", "select * from ns.tbl") - .withDefaultNamespace(viewIdentifier.namespace()) - .create()) - .isInstanceOf(AlreadyExistsException.class) - .hasMessageStartingWith("View already exists: ns.view"); + assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should not exist").isFalse(); tableCatalog().buildTable(tableIdentifier, SCHEMA).create(); - assertThat(tableCatalog().tableExists(tableIdentifier)).isTrue(); - assertThat(catalog().viewExists(tableIdentifier)).isFalse(); + + assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should exist").isTrue(); assertThatThrownBy( () -> @@ -264,33 +237,26 @@ public void createTableThatAlreadyExistsAsView() { .as("Only valid for catalogs that support tables") .isNotNull(); - TableIdentifier viewOne = TableIdentifier.of("ns", "viewOne"); - TableIdentifier viewTwo = TableIdentifier.of("ns", "viewTwo"); + TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); if (requiresNamespaceCreate()) { - catalog().createNamespace(viewOne.namespace()); + catalog().createNamespace(viewIdentifier.namespace()); } - assertThat(catalog().viewExists(viewOne)).isFalse(); - assertThat(tableCatalog().tableExists(viewTwo)).isFalse(); - - for (TableIdentifier identifier : Arrays.asList(viewTwo, viewOne)) { - assertThat( - catalog() - .buildView(identifier) - .withSchema(SCHEMA) - .withDefaultNamespace(identifier.namespace()) - .withQuery("spark", "select * from ns.tbl") - .create()) - .isNotNull(); - - assertThat(catalog().viewExists(identifier)).isTrue(); - assertThat(tableCatalog().tableExists(identifier)).isFalse(); - } + assertThat(catalog().viewExists(viewIdentifier)).as("View should not exist").isFalse(); - assertThatThrownBy(() -> tableCatalog().buildTable(viewTwo, SCHEMA).create()) + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(viewIdentifier)).as("View should exist").isTrue(); + + assertThatThrownBy(() -> tableCatalog().buildTable(viewIdentifier, SCHEMA).create()) .isInstanceOf(AlreadyExistsException.class) - .hasMessageStartingWith("View with same name already exists: ns.viewTwo"); + .hasMessageStartingWith("View with same name already exists: ns.view"); } @Test @@ -302,6 +268,8 @@ public void renameView() { catalog().createNamespace(from.namespace()); } + assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); + catalog() .buildView(from) .withSchema(SCHEMA) @@ -310,44 +278,20 @@ public void renameView() { .create(); assertThat(catalog().viewExists(from)).as("View should exist").isTrue(); - assertThat(catalog().listViews(from.namespace())).containsExactly(from); catalog().renameView(from, to); - assertThat(catalog().listViews(to.namespace())).containsExactly(to); - assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); - assertThat(catalog().viewExists(to)).as("View should exist").isTrue(); - - View view = catalog().loadView(to); - assertThat(view).isNotNull(); + assertThat(catalog().viewExists(from)).as("View should not exist with old name").isFalse(); + assertThat(catalog().viewExists(to)).as("View should exist with new name").isTrue(); - // validate view settings - assertThat(view.name()).isEqualTo(catalog().name() + "." + to); - assertThat(view.properties()).isEmpty(); - assertThat(view.history()) - .hasSize(1) - .first() - .extracting(ViewHistoryEntry::versionId) + // ensure current view version id didn't change after renaming + assertThat(catalog().loadView(to)) + .isNotNull() + .extracting(View::currentVersion) + .extracting(ViewVersion::versionId) .isEqualTo(1); - assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); - assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); - assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); - - assertThat(view.currentVersion()) - .isEqualTo( - ImmutableViewVersion.builder() - .timestampMillis(view.currentVersion().timestampMillis()) - .versionId(1) - .schemaId(SCHEMA.schemaId()) - .putSummary("operation", "create") - .defaultNamespace(to.namespace()) - .addRepresentations( - ImmutableSQLViewRepresentation.builder() - .sql("select * from ns.tbl") - .dialect("spark") - .build()) - .build()); + assertThat(catalog().dropView(from)).isFalse(); assertThat(catalog().dropView(to)).isTrue(); assertThat(catalog().viewExists(to)).as("View should not exist").isFalse(); } @@ -362,6 +306,8 @@ public void renameViewUsingDifferentNamespace() { catalog().createNamespace(to.namespace()); } + assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); + catalog() .buildView(from) .withSchema(SCHEMA) @@ -369,44 +315,12 @@ public void renameViewUsingDifferentNamespace() { .withQuery("spark", "select * from ns.tbl") .create(); - assertThat(catalog().listViews(from.namespace())).containsExactly(from); assertThat(catalog().viewExists(from)).as("View should exist").isTrue(); catalog().renameView(from, to); - assertThat(catalog().listViews(to.namespace())).containsExactly(to); - assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); - assertThat(catalog().viewExists(to)).as("View should exist").isTrue(); - - View view = catalog().loadView(to); - assertThat(view).isNotNull(); - - // validate view settings - assertThat(view.name()).isEqualTo(catalog().name() + "." + to); - assertThat(view.properties()).isEmpty(); - assertThat(view.history()) - .hasSize(1) - .first() - .extracting(ViewHistoryEntry::versionId) - .isEqualTo(1); - assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); - assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); - assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); - - assertThat(view.currentVersion()) - .isEqualTo( - ImmutableViewVersion.builder() - .timestampMillis(view.currentVersion().timestampMillis()) - .versionId(1) - .schemaId(SCHEMA.schemaId()) - .putSummary("operation", "create") - .defaultNamespace(from.namespace()) - .addRepresentations( - ImmutableSQLViewRepresentation.builder() - .sql("select * from ns.tbl") - .dialect("spark") - .build()) - .build()); + assertThat(catalog().viewExists(from)).as("View should not exist with old name").isFalse(); + assertThat(catalog().viewExists(to)).as("View should exist with new name").isTrue(); assertThat(catalog().dropView(from)).isFalse(); assertThat(catalog().dropView(to)).isTrue(); @@ -422,6 +336,8 @@ public void renameViewNamespaceMissing() { catalog().createNamespace(from.namespace()); } + assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); + catalog() .buildView(from) .withSchema(SCHEMA) @@ -438,7 +354,7 @@ public void renameViewNamespaceMissing() { @Test public void renameViewSourceMissing() { - TableIdentifier from = TableIdentifier.of("ns", "view"); + TableIdentifier from = TableIdentifier.of("ns", "non_existing"); TableIdentifier to = TableIdentifier.of("ns", "renamedView"); if (requiresNamespaceCreate()) { @@ -450,40 +366,68 @@ public void renameViewSourceMissing() { assertThatThrownBy(() -> catalog().renameView(from, to)) .isInstanceOf(NoSuchViewException.class) .hasMessageContaining("View does not exist"); - - assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); - assertThat(catalog().viewExists(to)).as("View should not exist").isFalse(); } @Test - public void renameViewTargetAlreadyExists() { - TableIdentifier from = TableIdentifier.of("ns", "view"); - TableIdentifier to = TableIdentifier.of("ns", "renamedView"); + public void renameViewTargetAlreadyExistsAsView() { + TableIdentifier viewOne = TableIdentifier.of("ns", "viewOne"); + TableIdentifier viewTwo = TableIdentifier.of("ns", "viewTwo"); if (requiresNamespaceCreate()) { - catalog().createNamespace(from.namespace()); + catalog().createNamespace(viewOne.namespace()); } - for (TableIdentifier viewIdentifier : ImmutableList.of(from, to)) { + for (TableIdentifier identifier : ImmutableList.of(viewOne, viewTwo)) { + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + catalog() - .buildView(viewIdentifier) + .buildView(identifier) .withSchema(SCHEMA) - .withDefaultNamespace(from.namespace()) + .withDefaultNamespace(viewOne.namespace()) .withQuery("spark", "select * from ns.tbl") .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); } - assertThatThrownBy(() -> catalog().renameView(from, to)) + assertThatThrownBy(() -> catalog().renameView(viewOne, viewTwo)) .isInstanceOf(AlreadyExistsException.class) - .hasMessageContaining("Cannot rename ns.view to ns.renamedView. View already exists"); + .hasMessageContaining("Cannot rename ns.viewOne to ns.viewTwo. View already exists"); + } + + @Test + public void renameViewTargetAlreadyExistsAsTable() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); + TableIdentifier tableIdentifier = TableIdentifier.of("ns", "table"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(tableIdentifier.namespace()); + } + + assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should not exist").isFalse(); + + tableCatalog().buildTable(tableIdentifier, SCHEMA).create(); + + assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should exist").isTrue(); - // rename view where a table with the same name already exists - TableIdentifier identifier = TableIdentifier.of("ns", "tbl"); - tableCatalog().buildTable(identifier, SCHEMA).create(); + assertThat(catalog().viewExists(viewIdentifier)).as("View should not exist").isFalse(); - assertThatThrownBy(() -> catalog().renameView(from, identifier)) + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(viewIdentifier)).as("View should exist").isTrue(); + + assertThatThrownBy(() -> catalog().renameView(viewIdentifier, tableIdentifier)) .isInstanceOf(AlreadyExistsException.class) - .hasMessageContaining("Cannot rename ns.view to ns.tbl. Table already exists"); + .hasMessageContaining("Cannot rename ns.view to ns.table. Table already exists"); } @Test @@ -491,7 +435,6 @@ public void listViews() { Namespace ns1 = Namespace.of("ns1"); Namespace ns2 = Namespace.of("ns2"); - TableIdentifier tableIdentifier = TableIdentifier.of(ns1, "table"); TableIdentifier view1 = TableIdentifier.of(ns1, "view1"); TableIdentifier view2 = TableIdentifier.of(ns2, "view2"); TableIdentifier view3 = TableIdentifier.of(ns2, "view3"); @@ -501,12 +444,6 @@ public void listViews() { catalog().createNamespace(ns2); } - if (null != tableCatalog()) { - tableCatalog().buildTable(tableIdentifier, SCHEMA).create(); - assertThat(tableCatalog().listTables(ns1)).containsExactly(tableIdentifier); - assertThat(tableCatalog().listTables(ns2)).isEmpty(); - } - assertThat(catalog().listViews(ns1)).isEmpty(); assertThat(catalog().listViews(ns2)).isEmpty(); @@ -540,11 +477,6 @@ public void listViews() { assertThat(catalog().listViews(ns1)).containsExactly(view1); assertThat(catalog().listViews(ns2)).containsExactlyInAnyOrder(view2, view3); - if (null != tableCatalog()) { - assertThat(tableCatalog().listTables(ns1)).containsExactly(tableIdentifier); - assertThat(tableCatalog().listTables(ns2)).isEmpty(); - } - assertThat(catalog().dropView(view2)).isTrue(); assertThat(catalog().listViews(ns1)).containsExactly(view1); assertThat(catalog().listViews(ns2)).containsExactly(view3); @@ -558,6 +490,47 @@ public void listViews() { assertThat(catalog().listViews(ns2)).isEmpty(); } + @Test + public void listViewsAndTables() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + Namespace ns = Namespace.of("ns"); + + TableIdentifier tableIdentifier = TableIdentifier.of(ns, "table"); + TableIdentifier viewIdentifier = TableIdentifier.of(ns, "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(ns); + } + + assertThat(catalog().listViews(ns)).isEmpty(); + assertThat(tableCatalog().listTables(ns)).isEmpty(); + + tableCatalog().buildTable(tableIdentifier, SCHEMA).create(); + assertThat(catalog().listViews(ns)).isEmpty(); + assertThat(tableCatalog().listTables(ns)).containsExactly(tableIdentifier); + + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIdentifier.namespace()) + .withQuery("spark", "select * from ns1.tbl") + .create(); + + assertThat(catalog().listViews(ns)).containsExactly(viewIdentifier); + assertThat(tableCatalog().listTables(ns)).containsExactly(tableIdentifier); + + assertThat(tableCatalog().dropTable(tableIdentifier)).isTrue(); + assertThat(catalog().listViews(ns)).containsExactly(viewIdentifier); + assertThat(tableCatalog().listTables(ns)).isEmpty(); + + assertThat(catalog().dropView(viewIdentifier)).isTrue(); + assertThat(catalog().listViews(ns)).isEmpty(); + assertThat(tableCatalog().listTables(ns)).isEmpty(); + } + @ParameterizedTest(name = ".createOrReplace() = {arguments}") @ValueSource(booleans = {false, true}) public void createOrReplaceView(boolean useCreateOrReplace) { @@ -567,6 +540,8 @@ public void createOrReplaceView(boolean useCreateOrReplace) { catalog().createNamespace(identifier.namespace()); } + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + ViewBuilder viewBuilder = catalog() .buildView(identifier) @@ -580,8 +555,8 @@ public void createOrReplaceView(boolean useCreateOrReplace) { assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); // validate view settings - assertThat(view.name()).isEqualTo(catalog().name() + "." + identifier); - assertThat(view.properties()).isEqualTo(ImmutableMap.of("prop1", "val1", "prop2", "val2")); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalog().name(), identifier)); + assertThat(view.properties()).containsEntry("prop1", "val1").containsEntry("prop2", "val2"); assertThat(view.history()) .hasSize(1) .first() @@ -618,9 +593,8 @@ public void createOrReplaceView(boolean useCreateOrReplace) { View replacedView = useCreateOrReplace ? viewBuilder.createOrReplace() : viewBuilder.replace(); // validate replaced view settings - assertThat(replacedView.name()).isEqualTo(catalog().name() + "." + identifier); + assertThat(replacedView.name()).isEqualTo(ViewUtil.fullViewName(catalog().name(), identifier)); assertThat(replacedView.properties()) - .hasSize(4) .containsEntry("prop1", "val1") .containsEntry("prop2", "val2") .containsEntry("replacedProp1", "val1") @@ -681,13 +655,41 @@ public void updateViewProperties() { .withQuery("spark", "select * from ns.tbl") .create(); - assertThat(view.properties()).isEmpty(); ViewVersion viewVersion = view.currentVersion(); - assertThat(viewVersion.operation()).isEqualTo("create"); - assertThat(viewVersion.versionId()).isEqualTo(1); - assertThat(view.history()).hasSize(1); - assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); - assertThat(view.versions()).hasSize(1).containsExactly(viewVersion); + + view.updateProperties().set("key1", "val1").set("key2", "val2").remove("non-existing").commit(); + + View updatedView = catalog().loadView(identifier); + assertThat(updatedView.properties()) + .containsEntry("key1", "val1") + .containsEntry("key2", "val2"); + + // history and view versions should stay the same after updating view properties + assertThat(updatedView.history()).hasSize(1).isEqualTo(view.history()); + assertThat(updatedView.versions()).hasSize(1).containsExactly(viewVersion); + + assertThat(catalog().dropView(identifier)).isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + } + + @Test + public void updateViewPropertiesErrorCases() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); assertThatThrownBy( () -> catalog().loadView(identifier).updateProperties().set(null, "new-val1").commit()) @@ -716,57 +718,6 @@ public void updateViewProperties() { .commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot remove and update the same key: key2"); - - view.updateProperties().set("key1", "val1").set("key2", "val2").remove("non-existing").commit(); - - View updatedView = catalog().loadView(identifier); - assertThat(updatedView.properties()) - .hasSize(2) - .containsEntry("key1", "val1") - .containsEntry("key2", "val2"); - assertThat(updatedView.history()).hasSize(1).isEqualTo(view.history()); - assertThat(updatedView.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); - assertThat(updatedView.versions()).hasSize(1).containsExactly(viewVersion); - - // updating properties doesn't change the view version - ViewVersion updatedViewVersion = updatedView.currentVersion(); - assertThat(updatedViewVersion).isNotNull(); - assertThat(updatedViewVersion.versionId()).isEqualTo(viewVersion.versionId()); - assertThat(updatedViewVersion.summary()).isEqualTo(viewVersion.summary()); - assertThat(updatedViewVersion.operation()).isEqualTo(viewVersion.operation()); - - assertThatThrownBy( - () -> - catalog() - .loadView(identifier) - .updateProperties() - .set("key1", "new-val1") - .set("key3", "val3") - .remove("key2") - .set("key2", "new-val2") - .commit()) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot remove and update the same key: key2"); - - view.updateProperties().set("key1", "new-val1").set("key3", "val3").remove("key2").commit(); - - View updatedView2 = catalog().loadView(identifier); - assertThat(updatedView2.properties()) - .hasSize(2) - .containsEntry("key1", "new-val1") - .containsEntry("key3", "val3"); - assertThat(updatedView2.history()).hasSize(1).isEqualTo(view.history()); - assertThat(updatedView2.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); - assertThat(updatedView2.versions()).hasSize(1).containsExactly(viewVersion); - - ViewVersion updatedViewVersion2 = updatedView2.currentVersion(); - assertThat(updatedViewVersion2).isNotNull(); - assertThat(updatedViewVersion2.versionId()).isEqualTo(viewVersion.versionId()); - assertThat(updatedViewVersion2.summary()).isEqualTo(viewVersion.summary()); - assertThat(updatedViewVersion2.operation()).isEqualTo(viewVersion.operation()); - - assertThat(catalog().dropView(identifier)).isTrue(); - assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); } @Test @@ -800,24 +751,12 @@ public void replaceViewVersion() { .withQuery(spark.dialect(), spark.sql()) .create(); + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + ViewVersion viewVersion = view.currentVersion(); - assertThat(view.properties()).isEmpty(); - assertThat(view.history()) - .hasSize(1) - .first() - .extracting(ViewHistoryEntry::versionId) - .isEqualTo(viewVersion.versionId()); - assertThat(view.history()) - .hasSize(1) - .first() - .extracting(ViewHistoryEntry::versionId) - .isEqualTo(view.currentVersion().versionId()); - assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); - assertThat(viewVersion.operation()).isEqualTo("create"); - assertThat(viewVersion.versionId()).isEqualTo(1); assertThat(viewVersion.representations()).hasSize(2).containsExactly(trino, spark); - assertThat(view.versions()).hasSize(1).containsExactly(viewVersion); + // uses a different schema and view representation view.replaceVersion() .withSchema(OTHER_SCHEMA) .withQuery(trino.dialect(), trino.sql()) @@ -825,15 +764,14 @@ public void replaceViewVersion() { .withDefaultNamespace(identifier.namespace()) .commit(); + // history and view versions should reflect the changes View updatedView = catalog().loadView(identifier); - assertThat(updatedView.properties()).isEmpty(); assertThat(updatedView.history()) .hasSize(2) .element(0) .extracting(ViewHistoryEntry::versionId) .isEqualTo(viewVersion.versionId()); assertThat(updatedView.history()) - .hasSize(2) .element(1) .extracting(ViewHistoryEntry::versionId) .isEqualTo(updatedView.currentVersion().versionId()); @@ -855,47 +793,71 @@ public void replaceViewVersion() { assertThat(updatedViewVersion.defaultCatalog()).isEqualTo("default"); assertThat(updatedViewVersion.defaultNamespace()).isEqualTo(identifier.namespace()); + assertThat(catalog().dropView(identifier)).isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + } + + @Test + public void replaceViewVersionByUpdatingSqlForDialect() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + SQLViewRepresentation spark = + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("spark") + .build(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery(spark.dialect(), spark.sql()) + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + + ViewVersion viewVersion = view.currentVersion(); + assertThat(viewVersion.representations()).hasSize(1).containsExactly(spark); + SQLViewRepresentation updatedSpark = ImmutableSQLViewRepresentation.builder() .sql("select * from ns.updated_tbl") .dialect("spark") .build(); + // only update the SQL for spark view.replaceVersion() - .withQuery(updatedSpark.dialect(), updatedSpark.sql()) + .withSchema(SCHEMA) .withDefaultNamespace(identifier.namespace()) - .withSchema(OTHER_SCHEMA) + .withQuery(updatedSpark.dialect(), updatedSpark.sql()) .commit(); - View updatedView2 = catalog().loadView(identifier); - assertThat(updatedView2.properties()).isEmpty(); - assertThat(updatedView2.history()) - .hasSize(3) + // history and view versions should reflect the changes + View updatedView = catalog().loadView(identifier); + assertThat(updatedView.history()) + .hasSize(2) .element(0) .extracting(ViewHistoryEntry::versionId) .isEqualTo(viewVersion.versionId()); - assertThat(updatedView2.history()) + assertThat(updatedView.history()) .element(1) .extracting(ViewHistoryEntry::versionId) - .isEqualTo(updatedViewVersion.versionId()); - assertThat(updatedView2.history()) - .element(2) - .extracting(ViewHistoryEntry::versionId) - .isEqualTo(updatedView2.currentVersion().versionId()); - assertThat(updatedView.schemas()) + .isEqualTo(updatedView.currentVersion().versionId()); + assertThat(updatedView.versions()) .hasSize(2) - .containsKey(SCHEMA.schemaId()) - .containsKey(OTHER_SCHEMA.schemaId()); - assertThat(updatedView2.versions()) - .hasSize(3) - .containsExactly(viewVersion, updatedViewVersion, updatedView2.currentVersion()); + .containsExactly(viewVersion, updatedView.currentVersion()); - ViewVersion updatedViewVersion2 = updatedView2.currentVersion(); - assertThat(updatedViewVersion2).isNotNull(); - assertThat(updatedViewVersion2.versionId()).isEqualTo(updatedViewVersion.versionId() + 1); - assertThat(updatedViewVersion2.summary()).hasSize(1).containsEntry("operation", "replace"); - assertThat(updatedViewVersion2.operation()).isEqualTo("replace"); - assertThat(updatedViewVersion2.representations()).hasSize(1).containsExactly(updatedSpark); + // updated view should have the new SQL + assertThat(updatedView.currentVersion().representations()) + .hasSize(1) + .containsExactly(updatedSpark); assertThat(catalog().dropView(identifier)).isTrue(); assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); @@ -925,6 +887,8 @@ public void replaceViewVersionErrorCases() { .withQuery(trino.dialect(), trino.sql()) .create(); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isTrue(); + // empty commits are not allowed assertThatThrownBy(() -> view.replaceVersion().commit()) .isInstanceOf(IllegalStateException.class) From 87cbd193c47c1e797b62a8ee00597de95106f223 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 25 Sep 2023 11:18:23 +0200 Subject: [PATCH 03/14] review feedback --- .../view/BaseMetastoreViewCatalog.java | 26 ++- .../iceberg/view/ViewVersionReplace.java | 8 +- .../apache/iceberg/view/ViewCatalogTests.java | 165 ++++++++++++++---- 3 files changed, 157 insertions(+), 42 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java index f6d0da45b623..0ec2b861de95 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.view; +import java.util.List; import java.util.Map; import org.apache.iceberg.BaseMetastoreCatalog; import org.apache.iceberg.Schema; @@ -28,6 +29,7 @@ import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.NoSuchViewException; 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; public abstract class BaseMetastoreViewCatalog extends BaseMetastoreCatalog implements ViewCatalog { @@ -66,7 +68,9 @@ protected class BaseViewBuilder implements ViewBuilder { private final TableIdentifier identifier; private final ImmutableViewVersion.Builder viewVersionBuilder = ImmutableViewVersion.builder(); private final Map properties = Maps.newHashMap(); - private Schema schema; + private final List representations = Lists.newArrayList(); + private Namespace defaultNamespace = null; + private Schema schema = null; protected BaseViewBuilder(TableIdentifier identifier) { Preconditions.checkArgument( @@ -83,7 +87,7 @@ public ViewBuilder withSchema(Schema newSchema) { @Override public ViewBuilder withQuery(String dialect, String sql) { - viewVersionBuilder.addRepresentations( + representations.add( ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build()); return this; } @@ -96,7 +100,7 @@ public ViewBuilder withDefaultCatalog(String defaultCatalog) { @Override public ViewBuilder withDefaultNamespace(Namespace namespace) { - viewVersionBuilder.defaultNamespace(namespace); + this.defaultNamespace = namespace; return this; } @@ -137,9 +141,17 @@ private View create(ViewOperations ops) { throw new AlreadyExistsException("View already exists: %s", identifier); } + Preconditions.checkState( + !representations.isEmpty(), "Cannot create view without specifying a query"); + Preconditions.checkState(null != schema, "Cannot create view without specifying schema"); + Preconditions.checkState( + null != defaultNamespace, "Cannot create view without specifying a default namespace"); + ViewVersion viewVersion = viewVersionBuilder .versionId(1) + .addAllRepresentations(representations) + .defaultNamespace(defaultNamespace) .timestampMillis(System.currentTimeMillis()) .putSummary("operation", "create") .build(); @@ -165,6 +177,12 @@ private View replace(ViewOperations ops) { throw new NoSuchViewException("View does not exist: %s", identifier); } + Preconditions.checkState( + !representations.isEmpty(), "Cannot replace view without specifying a query"); + Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); + Preconditions.checkState( + null != defaultNamespace, "Cannot replace view without specifying a default namespace"); + ViewMetadata metadata = ops.current(); int maxVersionId = metadata.versions().stream() @@ -175,6 +193,8 @@ private View replace(ViewOperations ops) { ViewVersion viewVersion = viewVersionBuilder .versionId(maxVersionId + 1) + .addAllRepresentations(representations) + .defaultNamespace(defaultNamespace) .timestampMillis(System.currentTimeMillis()) .putSummary("operation", "replace") .build(); diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java index 387771f64b2d..b5f2fd8e5a3d 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java @@ -40,9 +40,9 @@ class ViewVersionReplace implements ReplaceViewVersion { private final ViewOperations ops; private final List representations = Lists.newArrayList(); private ViewMetadata base; - private Namespace defaultNamespace; - private String defaultCatalog; - private Schema schema; + private Namespace defaultNamespace = null; + private String defaultCatalog = null; + private Schema schema = null; ViewVersionReplace(ViewOperations ops) { this.ops = ops; @@ -60,6 +60,8 @@ private ViewMetadata internalApply(ViewMetadata metadata) { Preconditions.checkState( !representations.isEmpty(), "Cannot replace view without specifying a query"); Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); + Preconditions.checkState( + null != defaultNamespace, "Cannot replace view without specifying a default namespace"); ViewVersion viewVersion = metadata.currentVersion(); int maxVersionId = diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 019040f81dfa..ddc35417e87b 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -39,14 +39,18 @@ import org.junit.jupiter.params.provider.ValueSource; public abstract class ViewCatalogTests { + // the schema ID of SCHEMA / OTHER_SCHEMA are by default set to 0. The schema id of SCHEMA will + // stay 0, but the schema id of OTHER_SCHEMA will be re-assigned to 1 + private static final int EXPECTED_SCHEMA_ID = 0; + private static final int EXPECTED_OTHER_SCHEMA_ID = 1; + protected static final Schema SCHEMA = new Schema( - 1, required(3, "id", Types.IntegerType.get(), "unique ID"), required(4, "data", Types.StringType.get())); private static final Schema OTHER_SCHEMA = - new Schema(2, required(1, "some_id", Types.IntegerType.get())); + new Schema(required(1, "some_id", Types.IntegerType.get())); protected abstract C catalog(); @@ -84,8 +88,9 @@ public void basicCreateView() { .first() .extracting(ViewHistoryEntry::versionId) .isEqualTo(1); - assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(view.schema().schemaId()).isEqualTo(EXPECTED_SCHEMA_ID); assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); + assertThat(view.schemas()).hasSize(1).containsKey(EXPECTED_SCHEMA_ID); assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); assertThat(view.currentVersion()) @@ -93,7 +98,7 @@ public void basicCreateView() { ImmutableViewVersion.builder() .timestampMillis(view.currentVersion().timestampMillis()) .versionId(1) - .schemaId(SCHEMA.schemaId()) + .schemaId(EXPECTED_SCHEMA_ID) .putSummary("operation", "create") .defaultNamespace(identifier.namespace()) .addRepresentations( @@ -139,9 +144,9 @@ public void completeCreateView() { .first() .extracting(ViewHistoryEntry::versionId) .isEqualTo(1); - assertThat(view.schema().schemaId()).isEqualTo(SCHEMA.schemaId()); + assertThat(view.schema().schemaId()).isEqualTo(EXPECTED_SCHEMA_ID); assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); - assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); + assertThat(view.schemas()).hasSize(1).containsKey(EXPECTED_SCHEMA_ID); assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); assertThat(view.currentVersion()) @@ -149,7 +154,7 @@ public void completeCreateView() { ImmutableViewVersion.builder() .timestampMillis(view.currentVersion().timestampMillis()) .versionId(1) - .schemaId(SCHEMA.schemaId()) + .schemaId(EXPECTED_SCHEMA_ID) .putSummary("operation", "create") .defaultNamespace(identifier.namespace()) .addRepresentations( @@ -168,6 +173,45 @@ public void completeCreateView() { assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); } + @Test + public void createViewErrorCases() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + SQLViewRepresentation trino = + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("trino") + .build(); + + // query is required + assertThatThrownBy(() -> catalog().buildView(identifier).create()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot create view without specifying a query"); + + // schema is required + assertThatThrownBy( + () -> catalog().buildView(identifier).withQuery(trino.dialect(), trino.sql()).create()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot create view without specifying schema"); + + // default namespace is required + assertThatThrownBy( + () -> + catalog() + .buildView(identifier) + .withQuery(trino.dialect(), trino.sql()) + .withSchema(SCHEMA) + .create()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot create view without specifying a default namespace"); + } + @Test public void createViewThatAlreadyExists() { TableIdentifier identifier = TableIdentifier.of("ns", "view"); @@ -554,28 +598,8 @@ public void createOrReplaceView(boolean useCreateOrReplace) { assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); - // validate view settings - assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalog().name(), identifier)); - assertThat(view.properties()).containsEntry("prop1", "val1").containsEntry("prop2", "val2"); - assertThat(view.history()) - .hasSize(1) - .first() - .extracting(ViewHistoryEntry::versionId) - .isEqualTo(1); - assertThat(view.schema().schemaId()).isEqualTo(SCHEMA.schemaId()); - assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); - assertThat(view.schemas()).hasSize(1).containsKey(SCHEMA.schemaId()); - assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); - ViewVersion viewVersion = view.currentVersion(); - assertThat(viewVersion).isNotNull(); - assertThat(viewVersion.versionId()).isEqualTo(1); - assertThat(viewVersion.schemaId()).isEqualTo(SCHEMA.schemaId()); - assertThat(viewVersion.summary()).hasSize(1).containsEntry("operation", "create"); - assertThat(viewVersion.operation()).isEqualTo("create"); - assertThat(viewVersion.defaultNamespace()).isEqualTo(identifier.namespace()); assertThat(viewVersion.representations()) - .hasSize(1) .containsExactly( ImmutableSQLViewRepresentation.builder() .sql("select * from ns.tbl") @@ -609,12 +633,12 @@ public void createOrReplaceView(boolean useCreateOrReplace) { .extracting(ViewHistoryEntry::versionId) .isEqualTo(2); - assertThat(replacedView.schema().schemaId()).isEqualTo(OTHER_SCHEMA.schemaId()); + assertThat(replacedView.schema().schemaId()).isEqualTo(EXPECTED_OTHER_SCHEMA_ID); assertThat(replacedView.schema().asStruct()).isEqualTo(OTHER_SCHEMA.asStruct()); assertThat(replacedView.schemas()) .hasSize(2) - .containsKey(SCHEMA.schemaId()) - .containsKey(OTHER_SCHEMA.schemaId()); + .containsKey(EXPECTED_SCHEMA_ID) + .containsKey(EXPECTED_OTHER_SCHEMA_ID); ViewVersion replacedViewVersion = replacedView.currentVersion(); assertThat(replacedView.versions()) @@ -622,11 +646,10 @@ public void createOrReplaceView(boolean useCreateOrReplace) { .containsExactly(viewVersion, replacedViewVersion); assertThat(replacedViewVersion).isNotNull(); assertThat(replacedViewVersion.versionId()).isEqualTo(2); - assertThat(replacedViewVersion.schemaId()).isEqualTo(OTHER_SCHEMA.schemaId()); + assertThat(replacedViewVersion.schemaId()).isEqualTo(EXPECTED_OTHER_SCHEMA_ID); assertThat(replacedViewVersion.operation()).isEqualTo("replace"); assertThat(replacedViewVersion.summary()).hasSize(1).containsEntry("operation", "replace"); assertThat(replacedViewVersion.representations()) - .hasSize(1) .containsExactly( ImmutableSQLViewRepresentation.builder() .sql("select count(*) from ns.tbl") @@ -637,6 +660,66 @@ public void createOrReplaceView(boolean useCreateOrReplace) { assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); } + @Test + public void replaceViewErrorCases() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + SQLViewRepresentation trino = + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("trino") + .build(); + + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery(trino.dialect(), trino.sql()) + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isTrue(); + + // query is required + assertThatThrownBy(() -> catalog().buildView(identifier).replace()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot replace view without specifying a query"); + + // schema is required + assertThatThrownBy( + () -> catalog().buildView(identifier).withQuery(trino.dialect(), trino.sql()).replace()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot replace view without specifying schema"); + + // default namespace is required + assertThatThrownBy( + () -> + catalog() + .buildView(identifier) + .withQuery(trino.dialect(), trino.sql()) + .withSchema(SCHEMA) + .replace()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot replace view without specifying a default namespace"); + + // cannot replace non-existing view + assertThatThrownBy( + () -> + catalog() + .buildView(TableIdentifier.of("ns", "non_existing")) + .withQuery(trino.dialect(), trino.sql()) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .replace()) + .isInstanceOf(NoSuchViewException.class) + .hasMessageStartingWith("View does not exist: ns.non_existing"); + } + @Test public void updateViewProperties() { TableIdentifier identifier = TableIdentifier.of("ns", "view"); @@ -777,8 +860,8 @@ public void replaceViewVersion() { .isEqualTo(updatedView.currentVersion().versionId()); assertThat(updatedView.schemas()) .hasSize(2) - .containsKey(SCHEMA.schemaId()) - .containsKey(OTHER_SCHEMA.schemaId()); + .containsKey(EXPECTED_SCHEMA_ID) + .containsKey(EXPECTED_OTHER_SCHEMA_ID); assertThat(updatedView.versions()) .hasSize(2) .containsExactly(viewVersion, updatedView.currentVersion()); @@ -789,7 +872,7 @@ public void replaceViewVersion() { assertThat(updatedViewVersion.summary()).hasSize(1).containsEntry("operation", "replace"); assertThat(updatedViewVersion.operation()).isEqualTo("replace"); assertThat(updatedViewVersion.representations()).hasSize(1).containsExactly(trino); - assertThat(updatedViewVersion.schemaId()).isEqualTo(OTHER_SCHEMA.schemaId()); + assertThat(updatedViewVersion.schemaId()).isEqualTo(EXPECTED_OTHER_SCHEMA_ID); assertThat(updatedViewVersion.defaultCatalog()).isEqualTo("default"); assertThat(updatedViewVersion.defaultNamespace()).isEqualTo(identifier.namespace()); @@ -798,7 +881,7 @@ public void replaceViewVersion() { } @Test - public void replaceViewVersionByUpdatingSqlForDialect() { + public void replaceViewVersionByUpdatingSQLForDialect() { TableIdentifier identifier = TableIdentifier.of("ns", "view"); if (requiresNamespaceCreate()) { @@ -903,5 +986,15 @@ public void replaceViewVersionErrorCases() { .commit()) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot replace view without specifying schema"); + + // default namespace is required + assertThatThrownBy( + () -> + view.replaceVersion() + .withQuery(trino.dialect(), trino.sql()) + .withSchema(SCHEMA) + .commit()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot replace view without specifying a default namespace"); } } From 4c2d734fbb4ef3d780eb0023098e99f8f8f2c493 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 25 Sep 2023 12:42:38 +0200 Subject: [PATCH 04/14] tx tests --- .../apache/iceberg/view/ViewCatalogTests.java | 97 +++++++++++++++++++ 1 file changed, 97 insertions(+) diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index ddc35417e87b..4a216ee48962 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.apache.iceberg.Schema; +import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; @@ -30,6 +31,7 @@ import org.apache.iceberg.catalog.ViewCatalog; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; @@ -303,6 +305,101 @@ public void createTableThatAlreadyExistsAsView() { .hasMessageStartingWith("View with same name already exists: ns.view"); } + @Test + public void createTableViaTransactionThatAlreadyExistsAsView() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(viewIdentifier.namespace()); + } + + assertThat(catalog().viewExists(viewIdentifier)).as("View should not exist").isFalse(); + + Transaction transaction = tableCatalog().buildTable(viewIdentifier, SCHEMA).createTransaction(); + + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(viewIdentifier)).as("View should exist").isTrue(); + + assertThatThrownBy(transaction::commitTransaction) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View with same name already exists: ns.view"); + } + + @Test + public void replaceTableViaTransactionThatAlreadyExistsAsView() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(viewIdentifier.namespace()); + } + + assertThat(catalog().viewExists(viewIdentifier)).as("View should not exist").isFalse(); + + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(viewIdentifier)).as("View should exist").isTrue(); + + // replace transaction requires table existence + assertThatThrownBy( + () -> + tableCatalog() + .buildTable(viewIdentifier, SCHEMA) + .replaceTransaction() + .commitTransaction()) + .isInstanceOf(NoSuchTableException.class) + .hasMessageStartingWith("Table does not exist: ns.view"); + } + + @Test + public void replaceViewThatAlreadyExistsAsTable() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + TableIdentifier tableIdentifier = TableIdentifier.of("ns", "table"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(tableIdentifier.namespace()); + } + + assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should not exist").isFalse(); + + tableCatalog().buildTable(tableIdentifier, SCHEMA).create(); + + assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should exist").isTrue(); + + // replace view requires the view to exist + assertThatThrownBy( + () -> + catalog() + .buildView(tableIdentifier) + .withSchema(OTHER_SCHEMA) + .withDefaultNamespace(tableIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .replace()) + .isInstanceOf(NoSuchViewException.class) + .hasMessageStartingWith("View does not exist: ns.table"); + } + @Test public void renameView() { TableIdentifier from = TableIdentifier.of("ns", "view"); From 305e62fbeae5c42f73f38fdea0f2ff21b4e7d682 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 25 Sep 2023 19:22:09 +0200 Subject: [PATCH 05/14] add some tests where view gets modified concurrently --- .../apache/iceberg/view/ViewCatalogTests.java | 138 +++++++++++++++++- 1 file changed, 136 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 4a216ee48962..9cb557dcaa22 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -30,6 +30,7 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.catalog.ViewCatalog; import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; @@ -780,7 +781,7 @@ public void replaceViewErrorCases() { .withQuery(trino.dialect(), trino.sql()) .create(); - assertThat(catalog().viewExists(identifier)).as("View should not exist").isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); // query is required assertThatThrownBy(() -> catalog().buildView(identifier).replace()) @@ -1067,7 +1068,7 @@ public void replaceViewVersionErrorCases() { .withQuery(trino.dialect(), trino.sql()) .create(); - assertThat(catalog().viewExists(identifier)).as("View should not exist").isTrue(); + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); // empty commits are not allowed assertThatThrownBy(() -> view.replaceVersion().commit()) @@ -1094,4 +1095,137 @@ public void replaceViewVersionErrorCases() { .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot replace view without specifying a default namespace"); } + + @Test + public void updateViewPropertiesConflict() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("trino", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + UpdateViewProperties updateViewProperties = view.updateProperties(); + + // drop view and then try to use the updateProperties API + catalog().dropView(identifier); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + assertThatThrownBy(() -> updateViewProperties.set("key1", "val1").commit()) + .isInstanceOf(CommitFailedException.class) + .hasMessageContaining("Cannot commit"); + } + + @Test + public void replaceViewVersionConflict() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("trino", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + ReplaceViewVersion replaceViewVersion = view.replaceVersion(); + + // drop view and then try to use the replaceVersion API + catalog().dropView(identifier); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + assertThatThrownBy( + () -> + replaceViewVersion + .withQuery("trino", "select * from ns.tbl") + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .commit()) + .isInstanceOf(CommitFailedException.class) + .hasMessageContaining("Cannot commit"); + } + + @Test + public void createViewConflict() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + ViewBuilder viewBuilder = catalog().buildView(identifier); + + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("trino", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + + // the view was already created concurrently + assertThatThrownBy( + () -> + viewBuilder + .withQuery("trino", "select * from ns.tbl") + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .create()) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("View already exists: ns.view"); + } + + @Test + public void replaceViewConflict() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("trino", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + ViewBuilder viewBuilder = catalog().buildView(identifier); + + catalog().dropView(identifier); + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + // the view was already dropped concurrently + assertThatThrownBy( + () -> + viewBuilder + .withQuery("trino", "select * from ns.tbl") + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .replace()) + .isInstanceOf(NoSuchViewException.class) + .hasMessageStartingWith("View does not exist: ns.view"); + } } From 7c176862730007a407355425512080aa0423c1b7 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 26 Sep 2023 09:26:00 +0200 Subject: [PATCH 06/14] move refresh into internalApply() --- .../apache/iceberg/view/PropertiesUpdate.java | 15 ++++++--------- .../apache/iceberg/view/ViewVersionReplace.java | 16 ++++++++-------- 2 files changed, 14 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/PropertiesUpdate.java b/core/src/main/java/org/apache/iceberg/view/PropertiesUpdate.java index 8b2142c069ab..48bcfc3a6805 100644 --- a/core/src/main/java/org/apache/iceberg/view/PropertiesUpdate.java +++ b/core/src/main/java/org/apache/iceberg/view/PropertiesUpdate.java @@ -49,16 +49,13 @@ class PropertiesUpdate implements UpdateViewProperties { @Override public Map apply() { - this.base = ops.refresh(); - - return internalApply(base).properties(); + return internalApply().properties(); } - private ViewMetadata internalApply(ViewMetadata metadata) { - return ViewMetadata.buildFrom(metadata) - .setProperties(updates) - .removeProperties(removals) - .build(); + private ViewMetadata internalApply() { + this.base = ops.refresh(); + + return ViewMetadata.buildFrom(base).setProperties(updates).removeProperties(removals).build(); } @Override @@ -76,7 +73,7 @@ public void commit() { base.properties(), COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), 2.0 /* exponential */) .onlyRetryOn(CommitFailedException.class) - .run(taskOps -> taskOps.commit(base, internalApply(base))); + .run(taskOps -> taskOps.commit(base, internalApply())); } @Override diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java index b5f2fd8e5a3d..15f05c531d07 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java @@ -51,21 +51,21 @@ class ViewVersionReplace implements ReplaceViewVersion { @Override public ViewVersion apply() { - this.base = ops.refresh(); - - return internalApply(base).currentVersion(); + return internalApply().currentVersion(); } - private ViewMetadata internalApply(ViewMetadata metadata) { + private ViewMetadata internalApply() { Preconditions.checkState( !representations.isEmpty(), "Cannot replace view without specifying a query"); Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); Preconditions.checkState( null != defaultNamespace, "Cannot replace view without specifying a default namespace"); - ViewVersion viewVersion = metadata.currentVersion(); + this.base = ops.refresh(); + + ViewVersion viewVersion = base.currentVersion(); int maxVersionId = - metadata.versions().stream() + base.versions().stream() .map(ViewVersion::versionId) .max(Integer::compareTo) .orElseGet(viewVersion::versionId); @@ -81,7 +81,7 @@ private ViewMetadata internalApply(ViewMetadata metadata) { .addAllRepresentations(representations) .build(); - return ViewMetadata.buildFrom(metadata).setCurrentVersion(newVersion, schema).build(); + return ViewMetadata.buildFrom(base).setCurrentVersion(newVersion, schema).build(); } @Override @@ -99,7 +99,7 @@ public void commit() { base.properties(), COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), 2.0 /* exponential */) .onlyRetryOn(CommitFailedException.class) - .run(taskOps -> taskOps.commit(base, internalApply(base))); + .run(taskOps -> taskOps.commit(base, internalApply())); } @Override From b776333a392f580895c8c112e4a0fc3887609c60 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 26 Sep 2023 09:36:42 +0200 Subject: [PATCH 07/14] check view metadata after renaming --- .../apache/iceberg/view/ViewCatalogTests.java | 49 ++++++++++++------- 1 file changed, 31 insertions(+), 18 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 9cb557dcaa22..e1e5bf9e730d 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -412,26 +412,29 @@ public void renameView() { assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); - catalog() - .buildView(from) - .withSchema(SCHEMA) - .withDefaultNamespace(from.namespace()) - .withQuery("spark", "select * from ns.tbl") - .create(); + View view = + catalog() + .buildView(from) + .withSchema(SCHEMA) + .withDefaultNamespace(from.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); assertThat(catalog().viewExists(from)).as("View should exist").isTrue(); + ViewMetadata original = ((BaseView) view).operations().current(); + catalog().renameView(from, to); assertThat(catalog().viewExists(from)).as("View should not exist with old name").isFalse(); assertThat(catalog().viewExists(to)).as("View should exist with new name").isTrue(); - // ensure current view version id didn't change after renaming - assertThat(catalog().loadView(to)) - .isNotNull() - .extracting(View::currentVersion) - .extracting(ViewVersion::versionId) - .isEqualTo(1); + // ensure view metadata didn't change after renaming + View renamed = catalog().loadView(to); + assertThat(((BaseView) renamed).operations().current()) + .usingRecursiveComparison() + .ignoringFieldsOfTypes(Schema.class) + .isEqualTo(original); assertThat(catalog().dropView(from)).isFalse(); assertThat(catalog().dropView(to)).isTrue(); @@ -450,20 +453,30 @@ public void renameViewUsingDifferentNamespace() { assertThat(catalog().viewExists(from)).as("View should not exist").isFalse(); - catalog() - .buildView(from) - .withSchema(SCHEMA) - .withDefaultNamespace(from.namespace()) - .withQuery("spark", "select * from ns.tbl") - .create(); + View view = + catalog() + .buildView(from) + .withSchema(SCHEMA) + .withDefaultNamespace(from.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); assertThat(catalog().viewExists(from)).as("View should exist").isTrue(); + ViewMetadata original = ((BaseView) view).operations().current(); + catalog().renameView(from, to); assertThat(catalog().viewExists(from)).as("View should not exist with old name").isFalse(); assertThat(catalog().viewExists(to)).as("View should exist with new name").isTrue(); + // ensure view metadata didn't change after renaming + View renamed = catalog().loadView(to); + assertThat(((BaseView) renamed).operations().current()) + .usingRecursiveComparison() + .ignoringFieldsOfTypes(Schema.class) + .isEqualTo(original); + assertThat(catalog().dropView(from)).isFalse(); assertThat(catalog().dropView(to)).isTrue(); assertThat(catalog().viewExists(to)).as("View should not exist").isFalse(); From 371e615b8894b9d1d32de63943acdd853c54b2e2 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 24 Sep 2023 20:45:24 +0200 Subject: [PATCH 08/14] Core: Check for SQL with same dialect in View representations --- .../org/apache/iceberg/view/ViewMetadata.java | 18 ++++ .../apache/iceberg/view/TestViewMetadata.java | 32 ++++++++ .../iceberg/view/TestViewMetadataParser.java | 82 +++++++++++++++++++ .../apache/iceberg/view/ViewCatalogTests.java | 39 +++++++++ .../ViewMetadataMultipleSQLsForDialect.json | 63 ++++++++++++++ 5 files changed, 234 insertions(+) create mode 100644 core/src/test/resources/org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json diff --git a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java index 43dba2f6b2ed..6b879b2486dc 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java @@ -272,6 +272,11 @@ private int addVersionInternal(ViewVersion version) { "Cannot add version with unknown schema: %s", version.schemaId()); + for (Map.Entry entry : sqlDialectFrequency(version).entrySet()) { + Preconditions.checkArgument( + entry.getValue() == 1L, "Cannot add multiple SQLs for dialect: %s", entry.getKey()); + } + ViewVersion newVersion; if (newVersionId != version.versionId()) { newVersion = ImmutableViewVersion.builder().from(version).versionId(newVersionId).build(); @@ -293,6 +298,19 @@ private int addVersionInternal(ViewVersion version) { return newVersionId; } + /** + * @param version The view version to analyze + * @return A map of SQL dialect to its frequency in the list of representations for the given + * view version + */ + private Map sqlDialectFrequency(ViewVersion version) { + return version.representations().stream() + .filter(v -> v instanceof SQLViewRepresentation) + .map(v -> (SQLViewRepresentation) v) + .map(SQLViewRepresentation::dialect) + .collect(Collectors.groupingBy(d -> d, Collectors.counting())); + } + private int reuseOrCreateNewViewVersionId(ViewVersion viewVersion) { // if the view version already exists, use its id; otherwise use the highest id + 1 int newVersionId = viewVersion.versionId(); diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java index 4706640f5388..33b7261632e4 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java @@ -727,4 +727,36 @@ public void viewVersionAndSchemaDeduplication() { .containsExactly(schemaOne.asStruct(), schemaTwo.asStruct(), schemaThree.asStruct()); assertThat(viewMetadata.schemasById().keySet()).containsExactly(0, 1, 2); } + + @Test + public void viewMetadataWithMultipleSQLForSameDialect() { + assertThatThrownBy( + () -> + ViewMetadata.builder() + .setLocation("custom-location") + .addSchema( + new Schema(1, Types.NestedField.required(1, "x", Types.LongType.get()))) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(1) + .timestampMillis(23L) + .putSummary("operation", "create") + .defaultNamespace(Namespace.of("ns")) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .dialect("spark") + .sql("select * from ns.tbl") + .build()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .dialect("spark") + .sql("select * from ns.tbl2") + .build()) + .build()) + .setCurrentVersionId(1) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add multiple SQLs for dialect: spark"); + } } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java index 5fb9589522d5..a6ef14cef72f 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java @@ -26,6 +26,7 @@ import java.nio.file.Paths; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -226,4 +227,85 @@ public void viewMetadataWithMetadataLocation() throws Exception { .isEqualTo(expectedViewMetadata); assertThat(actual.metadataFileLocation()).isEqualTo(metadataLocation); } + + @Test + public void viewMetadataWithMultipleSQLsForDialectShouldBeReadable() throws Exception { + ViewVersion viewVersion = + ImmutableViewVersion.builder() + .versionId(1) + .timestampMillis(4353L) + .summary(ImmutableMap.of("operation", "create")) + .schemaId(1) + .defaultCatalog("some-catalog") + .defaultNamespace(Namespace.empty()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select 'foo' foo") + .dialect("spark-sql") + .build()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from foo") + .dialect("spark-sql") + .build()) + .build(); + + String json = + readViewMetadataInputFile( + "org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json"); + + // builder will throw an exception due to having multiple SQLs for the same dialect, thus + // construct the expected view metadata directly + ViewMetadata expectedViewMetadata = + ImmutableViewMetadata.of( + "fa6506c3-7681-40c8-86dc-e36561f83385", + 1, + "s3://bucket/test/location", + ImmutableList.of(TEST_SCHEMA), + 1, + ImmutableList.of(viewVersion), + ImmutableList.of( + ImmutableViewHistoryEntry.builder().versionId(1).timestampMillis(4353).build()), + ImmutableMap.of("some-key", "some-value"), + ImmutableList.of(), + null); + + // reading view metadata with multiple SQLs for the same dialects shouldn't fail + ViewMetadata actual = ViewMetadataParser.fromJson(json); + assertThat(actual) + .usingRecursiveComparison() + .ignoringFieldsOfTypes(Schema.class) + .isEqualTo(expectedViewMetadata); + } + + @Test + public void replaceViewMetadataWithMultipleSQLsForDialect() throws Exception { + String json = + readViewMetadataInputFile( + "org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json"); + + // reading view metadata with multiple SQLs for the same dialects shouldn't fail + ViewMetadata invalid = ViewMetadataParser.fromJson(json); + + // replace metadata with a new view version that fixes the SQL representations + ViewVersion viewVersion = + ImmutableViewVersion.builder() + .versionId(2) + .schemaId(1) + .timestampMillis(5555L) + .summary(ImmutableMap.of("operation", "replace")) + .defaultCatalog("some-catalog") + .defaultNamespace(Namespace.empty()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from foo") + .dialect("spark-sql") + .build()) + .build(); + + ViewMetadata replaced = + ViewMetadata.buildFrom(invalid).addVersion(viewVersion).setCurrentVersionId(2).build(); + + assertThat(replaced.currentVersion()).isEqualTo(viewVersion); + } } diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index e1e5bf9e730d..6f861eb090d8 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -213,6 +213,19 @@ public void createViewErrorCases() { .create()) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot create view without specifying a default namespace"); + + // cannot define multiple SQLs for same dialect + assertThatThrownBy( + () -> + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery(trino.dialect(), trino.sql()) + .withQuery(trino.dialect(), trino.sql()) + .create()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add multiple SQLs for dialect: trino"); } @Test @@ -829,6 +842,19 @@ public void replaceViewErrorCases() { .replace()) .isInstanceOf(NoSuchViewException.class) .hasMessageStartingWith("View does not exist: ns.non_existing"); + + // cannot define multiple SQLs for same dialect + assertThatThrownBy( + () -> + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery(trino.dialect(), trino.sql()) + .withQuery(trino.dialect(), trino.sql()) + .replace()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add multiple SQLs for dialect: trino"); } @Test @@ -1107,6 +1133,19 @@ public void replaceViewVersionErrorCases() { .commit()) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot replace view without specifying a default namespace"); + + // cannot define multiple SQLs for same dialect + assertThatThrownBy( + () -> + view.replaceVersion() + .withQuery(trino.dialect(), trino.sql()) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery(trino.dialect(), trino.sql()) + .withQuery(trino.dialect(), trino.sql()) + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add multiple SQLs for dialect: trino"); } @Test diff --git a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json new file mode 100644 index 000000000000..f849de31d0f1 --- /dev/null +++ b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json @@ -0,0 +1,63 @@ +{ + "view-uuid": "fa6506c3-7681-40c8-86dc-e36561f83385", + "format-version": 1, + "location": "s3://bucket/test/location", + "properties": {"some-key": "some-value"}, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "current-version-id": 1, + "versions": [ + { + "version-id": 1, + "timestamp-ms": 4353, + "summary": {"operation":"create"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 'foo' foo", + "dialect": "spark-sql" + }, + { + "type": "sql", + "sql": "select * from foo", + "dialect": "spark-sql" + } + ] + } + ], + "version-log": [ + { + "timestamp-ms": 4353, + "version-id": 1 + } + ] +} \ No newline at end of file From 252154e76bf135ffaf9716f9e9c8625a7073f9c9 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 25 Sep 2023 17:16:20 +0200 Subject: [PATCH 09/14] Core: Improve concurrency in InMemoryCatalog --- .../iceberg/inmemory/InMemoryCatalog.java | 258 ++++++++++-------- 1 file changed, 141 insertions(+), 117 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java index ee7666e9a7e9..566e56eec9b5 100644 --- a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java +++ b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java @@ -118,8 +118,10 @@ public boolean dropTable(TableIdentifier tableIdentifier, boolean purge) { lastMetadata = null; } - if (null == tables.remove(tableIdentifier)) { - return false; + synchronized (this) { + if (null == tables.remove(tableIdentifier)) { + return false; + } } if (purge && lastMetadata != null) { @@ -143,27 +145,29 @@ public List listTables(Namespace namespace) { } @Override - public synchronized void renameTable(TableIdentifier from, TableIdentifier to) { + public void renameTable(TableIdentifier from, TableIdentifier to) { if (from.equals(to)) { return; } - if (!namespaceExists(to.namespace())) { - throw new NoSuchNamespaceException( - "Cannot rename %s to %s. Namespace does not exist: %s", from, to, to.namespace()); - } + synchronized (this) { + if (!namespaceExists(to.namespace())) { + throw new NoSuchNamespaceException( + "Cannot rename %s to %s. Namespace does not exist: %s", from, to, to.namespace()); + } - String fromLocation = tables.get(from); - if (null == fromLocation) { - throw new NoSuchTableException("Cannot rename %s to %s. Table does not exist", from, to); - } + String fromLocation = tables.get(from); + if (null == fromLocation) { + throw new NoSuchTableException("Cannot rename %s to %s. Table does not exist", from, to); + } - if (tables.containsKey(to)) { - throw new AlreadyExistsException("Cannot rename %s to %s. Table already exists", from, to); - } + if (tables.containsKey(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. Table already exists", from, to); + } - tables.put(to, fromLocation); - tables.remove(from); + tables.put(to, fromLocation); + tables.remove(from); + } } @Override @@ -173,12 +177,14 @@ public void createNamespace(Namespace namespace) { @Override public void createNamespace(Namespace namespace, Map metadata) { - if (namespaceExists(namespace)) { - throw new AlreadyExistsException( - "Cannot create namespace %s. Namespace already exists", namespace); - } + synchronized (this) { + if (namespaceExists(namespace)) { + throw new AlreadyExistsException( + "Cannot create namespace %s. Namespace already exists", namespace); + } - namespaces.put(namespace, ImmutableMap.copyOf(metadata)); + namespaces.put(namespace, ImmutableMap.copyOf(metadata)); + } } @Override @@ -188,50 +194,59 @@ public boolean namespaceExists(Namespace namespace) { @Override public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException { - if (!namespaceExists(namespace)) { - return false; - } + synchronized (this) { + if (!namespaceExists(namespace)) { + return false; + } - List tableIdentifiers = listTables(namespace); - if (!tableIdentifiers.isEmpty()) { - throw new NamespaceNotEmptyException( - "Namespace %s is not empty. Contains %d table(s).", namespace, tableIdentifiers.size()); - } + List tableIdentifiers = listTables(namespace); + if (!tableIdentifiers.isEmpty()) { + throw new NamespaceNotEmptyException( + "Namespace %s is not empty. Contains %d table(s).", namespace, tableIdentifiers.size()); + } - return namespaces.remove(namespace) != null; + return namespaces.remove(namespace) != null; + } } @Override public boolean setProperties(Namespace namespace, Map properties) throws NoSuchNamespaceException { - if (!namespaceExists(namespace)) { - throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); - } + synchronized (this) { + if (!namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } - namespaces.computeIfPresent( - namespace, - (k, v) -> - ImmutableMap.builder().putAll(v).putAll(properties).buildKeepingLast()); + namespaces.computeIfPresent( + namespace, + (k, v) -> + ImmutableMap.builder() + .putAll(v) + .putAll(properties) + .buildKeepingLast()); - return true; + return true; + } } @Override public boolean removeProperties(Namespace namespace, Set properties) throws NoSuchNamespaceException { - if (!namespaceExists(namespace)) { - throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); - } + synchronized (this) { + if (!namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } - namespaces.computeIfPresent( - namespace, - (k, v) -> { - Map newProperties = Maps.newHashMap(v); - properties.forEach(newProperties::remove); - return ImmutableMap.copyOf(newProperties); - }); + namespaces.computeIfPresent( + namespace, + (k, v) -> { + Map newProperties = Maps.newHashMap(v); + properties.forEach(newProperties::remove); + return ImmutableMap.copyOf(newProperties); + }); - return true; + return true; + } } @Override @@ -308,35 +323,39 @@ protected InMemoryViewOperations newViewOps(TableIdentifier identifier) { @Override public boolean dropView(TableIdentifier identifier) { - return null != views.remove(identifier); + synchronized (this) { + return null != views.remove(identifier); + } } @Override - public synchronized void renameView(TableIdentifier from, TableIdentifier to) { + public void renameView(TableIdentifier from, TableIdentifier to) { if (from.equals(to)) { return; } - if (!namespaceExists(to.namespace())) { - throw new NoSuchNamespaceException( - "Cannot rename %s to %s. Namespace does not exist: %s", from, to, to.namespace()); - } + synchronized (this) { + if (!namespaceExists(to.namespace())) { + throw new NoSuchNamespaceException( + "Cannot rename %s to %s. Namespace does not exist: %s", from, to, to.namespace()); + } - String fromViewLocation = views.get(from); - if (null == fromViewLocation) { - throw new NoSuchViewException("Cannot rename %s to %s. View does not exist", from, to); - } + String fromViewLocation = views.get(from); + if (null == fromViewLocation) { + throw new NoSuchViewException("Cannot rename %s to %s. View does not exist", from, to); + } - if (tables.containsKey(to)) { - throw new AlreadyExistsException("Cannot rename %s to %s. Table already exists", from, to); - } + if (tables.containsKey(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. Table already exists", from, to); + } - if (views.containsKey(to)) { - throw new AlreadyExistsException("Cannot rename %s to %s. View already exists", from, to); - } + if (views.containsKey(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. View already exists", from, to); + } - views.put(to, fromViewLocation); - views.remove(from); + views.put(to, fromViewLocation); + views.remove(from); + } } private class InMemoryTableOperations extends BaseMetastoreTableOperations { @@ -361,35 +380,38 @@ public void doRefresh() { } @Override - public synchronized void doCommit(TableMetadata base, TableMetadata metadata) { + public void doCommit(TableMetadata base, TableMetadata metadata) { String newLocation = writeNewMetadataIfRequired(base == null, metadata); String oldLocation = base == null ? null : base.metadataFileLocation(); - if (null == base && !namespaceExists(tableIdentifier.namespace())) { - throw new NoSuchNamespaceException( - "Cannot create table %s. Namespace does not exist: %s", - tableIdentifier, tableIdentifier.namespace()); - } - - if (views.containsKey(tableIdentifier)) { - throw new AlreadyExistsException("View with same name already exists: %s", tableIdentifier); - } - - tables.compute( - tableIdentifier, - (k, existingLocation) -> { - if (!Objects.equal(existingLocation, oldLocation)) { - if (null == base) { - throw new AlreadyExistsException("Table already exists: %s", tableName()); + synchronized (InMemoryCatalog.this) { + if (null == base && !namespaceExists(tableIdentifier.namespace())) { + throw new NoSuchNamespaceException( + "Cannot create table %s. Namespace does not exist: %s", + tableIdentifier, tableIdentifier.namespace()); + } + + if (views.containsKey(tableIdentifier)) { + throw new AlreadyExistsException( + "View with same name already exists: %s", tableIdentifier); + } + + tables.compute( + tableIdentifier, + (k, existingLocation) -> { + if (!Objects.equal(existingLocation, oldLocation)) { + if (null == base) { + throw new AlreadyExistsException("Table already exists: %s", tableName()); + } + + throw new CommitFailedException( + "Cannot commit to table %s metadata location from %s to %s " + + "because it has been concurrently modified to %s", + tableIdentifier, oldLocation, newLocation, existingLocation); } - - throw new CommitFailedException( - "Cannot commit to table %s metadata location from %s to %s " - + "because it has been concurrently modified to %s", - tableIdentifier, oldLocation, newLocation, existingLocation); - } - return newLocation; - }); + return newLocation; + }); + } } @Override @@ -425,36 +447,38 @@ public void doRefresh() { } @Override - public synchronized void doCommit(ViewMetadata base, ViewMetadata metadata) { + public void doCommit(ViewMetadata base, ViewMetadata metadata) { String newLocation = writeNewMetadataIfRequired(metadata); String oldLocation = base == null ? null : currentMetadataLocation(); - if (null == base && !namespaceExists(identifier.namespace())) { - throw new NoSuchNamespaceException( - "Cannot create view %s. Namespace does not exist: %s", - identifier, identifier.namespace()); - } - - if (tables.containsKey(identifier)) { - throw new AlreadyExistsException("Table with same name already exists: %s", identifier); - } - - views.compute( - identifier, - (k, existingLocation) -> { - if (!Objects.equal(existingLocation, oldLocation)) { - if (null == base) { - throw new AlreadyExistsException("View already exists: %s", identifier); + synchronized (InMemoryCatalog.this) { + if (null == base && !namespaceExists(identifier.namespace())) { + throw new NoSuchNamespaceException( + "Cannot create view %s. Namespace does not exist: %s", + identifier, identifier.namespace()); + } + + if (tables.containsKey(identifier)) { + throw new AlreadyExistsException("Table with same name already exists: %s", identifier); + } + + views.compute( + identifier, + (k, existingLocation) -> { + if (!Objects.equal(existingLocation, oldLocation)) { + if (null == base) { + throw new AlreadyExistsException("View already exists: %s", identifier); + } + + throw new CommitFailedException( + "Cannot commit to view %s metadata location from %s to %s " + + "because it has been concurrently modified to %s", + identifier, oldLocation, newLocation, existingLocation); } - throw new CommitFailedException( - "Cannot commit to view %s metadata location from %s to %s " - + "because it has been concurrently modified to %s", - identifier, oldLocation, newLocation, existingLocation); - } - - return newLocation; - }); + return newLocation; + }); + } } @Override From b4d15738a3226a0c48a61e69843ab22390b2ca5a Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 27 Sep 2023 08:19:25 +0200 Subject: [PATCH 10/14] feedback --- .../org/apache/iceberg/view/ViewMetadata.java | 26 +++---- .../apache/iceberg/view/TestViewMetadata.java | 2 +- .../apache/iceberg/view/ViewCatalogTests.java | 73 ++++++++++++++++++- 3 files changed, 81 insertions(+), 20 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java index 6b879b2486dc..68619ddb2292 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java @@ -35,6 +35,7 @@ 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; import org.apache.iceberg.util.PropertyUtil; import org.immutables.value.Value; import org.immutables.value.Value.Style.ImplementationVisibility; @@ -272,9 +273,15 @@ private int addVersionInternal(ViewVersion version) { "Cannot add version with unknown schema: %s", version.schemaId()); - for (Map.Entry entry : sqlDialectFrequency(version).entrySet()) { - Preconditions.checkArgument( - entry.getValue() == 1L, "Cannot add multiple SQLs for dialect: %s", entry.getKey()); + Set dialects = Sets.newHashSet(); + for (ViewRepresentation repr : version.representations()) { + if (repr instanceof SQLViewRepresentation) { + SQLViewRepresentation sql = (SQLViewRepresentation) repr; + Preconditions.checkArgument( + dialects.add(sql.dialect()), + "Invalid view version: Cannot add multiple queries for dialect %s", + sql.dialect()); + } } ViewVersion newVersion; @@ -298,19 +305,6 @@ private int addVersionInternal(ViewVersion version) { return newVersionId; } - /** - * @param version The view version to analyze - * @return A map of SQL dialect to its frequency in the list of representations for the given - * view version - */ - private Map sqlDialectFrequency(ViewVersion version) { - return version.representations().stream() - .filter(v -> v instanceof SQLViewRepresentation) - .map(v -> (SQLViewRepresentation) v) - .map(SQLViewRepresentation::dialect) - .collect(Collectors.groupingBy(d -> d, Collectors.counting())); - } - private int reuseOrCreateNewViewVersionId(ViewVersion viewVersion) { // if the view version already exists, use its id; otherwise use the highest id + 1 int newVersionId = viewVersion.versionId(); diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java index 33b7261632e4..15c232e6f92d 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java @@ -757,6 +757,6 @@ public void viewMetadataWithMultipleSQLForSameDialect() { .setCurrentVersionId(1) .build()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot add multiple SQLs for dialect: spark"); + .hasMessage("Invalid view version: Cannot add multiple queries for dialect spark"); } } diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 6f861eb090d8..4eb2ae8ff10b 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -130,6 +130,7 @@ public void completeCreateView() { .buildView(identifier) .withSchema(SCHEMA) .withDefaultNamespace(identifier.namespace()) + .withDefaultCatalog(catalog().name()) .withQuery("spark", "select * from ns.tbl") .withQuery("trino", "select * from ns.tbl using X") .withProperty("prop1", "val1") @@ -160,6 +161,7 @@ public void completeCreateView() { .schemaId(EXPECTED_SCHEMA_ID) .putSummary("operation", "create") .defaultNamespace(identifier.namespace()) + .defaultCatalog(catalog().name()) .addRepresentations( ImmutableSQLViewRepresentation.builder() .sql("select * from ns.tbl") @@ -225,7 +227,7 @@ public void createViewErrorCases() { .withQuery(trino.dialect(), trino.sql()) .create()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot add multiple SQLs for dialect: trino"); + .hasMessage("Invalid view version: Cannot add multiple queries for dialect trino"); } @Test @@ -373,6 +375,7 @@ public void replaceTableViaTransactionThatAlreadyExistsAsView() { assertThat(catalog().viewExists(viewIdentifier)).as("View should exist").isTrue(); // replace transaction requires table existence + // TODO: replace should check whether the table exists as a view assertThatThrownBy( () -> tableCatalog() @@ -383,6 +386,39 @@ public void replaceTableViaTransactionThatAlreadyExistsAsView() { .hasMessageStartingWith("Table does not exist: ns.view"); } + @Test + public void createOrReplaceTableViaTransactionThatAlreadyExistsAsView() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(viewIdentifier.namespace()); + } + + assertThat(catalog().viewExists(viewIdentifier)).as("View should not exist").isFalse(); + + catalog() + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(viewIdentifier)).as("View should exist").isTrue(); + + assertThatThrownBy( + () -> + tableCatalog() + .buildTable(viewIdentifier, SCHEMA) + .createOrReplaceTransaction() + .commitTransaction()) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View with same name already exists: ns.view"); + } + @Test public void replaceViewThatAlreadyExistsAsTable() { Assumptions.assumeThat(tableCatalog()) @@ -402,6 +438,7 @@ public void replaceViewThatAlreadyExistsAsTable() { assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should exist").isTrue(); // replace view requires the view to exist + // TODO: replace should check whether the view exists as a table assertThatThrownBy( () -> catalog() @@ -414,6 +451,36 @@ public void replaceViewThatAlreadyExistsAsTable() { .hasMessageStartingWith("View does not exist: ns.table"); } + @Test + public void createOrReplaceViewThatAlreadyExistsAsTable() { + Assumptions.assumeThat(tableCatalog()) + .as("Only valid for catalogs that support tables") + .isNotNull(); + + TableIdentifier tableIdentifier = TableIdentifier.of("ns", "table"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(tableIdentifier.namespace()); + } + + assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should not exist").isFalse(); + + tableCatalog().buildTable(tableIdentifier, SCHEMA).create(); + + assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should exist").isTrue(); + + assertThatThrownBy( + () -> + catalog() + .buildView(tableIdentifier) + .withSchema(OTHER_SCHEMA) + .withDefaultNamespace(tableIdentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .createOrReplace()) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("Table with same name already exists: ns.table"); + } + @Test public void renameView() { TableIdentifier from = TableIdentifier.of("ns", "view"); @@ -854,7 +921,7 @@ public void replaceViewErrorCases() { .withQuery(trino.dialect(), trino.sql()) .replace()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot add multiple SQLs for dialect: trino"); + .hasMessage("Invalid view version: Cannot add multiple queries for dialect trino"); } @Test @@ -1145,7 +1212,7 @@ public void replaceViewVersionErrorCases() { .withQuery(trino.dialect(), trino.sql()) .commit()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot add multiple SQLs for dialect: trino"); + .hasMessage("Invalid view version: Cannot add multiple queries for dialect trino"); } @Test From 64cdef239a66a6d2bd896e4532e8efa6de7e997a Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 27 Sep 2023 18:12:11 +0200 Subject: [PATCH 11/14] use separate field for default catalog --- .../org/apache/iceberg/view/BaseMetastoreViewCatalog.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java index 0ec2b861de95..ca37be3eadd3 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java @@ -70,6 +70,7 @@ protected class BaseViewBuilder implements ViewBuilder { private final Map properties = Maps.newHashMap(); private final List representations = Lists.newArrayList(); private Namespace defaultNamespace = null; + private String defaultCatalog = null; private Schema schema = null; protected BaseViewBuilder(TableIdentifier identifier) { @@ -93,8 +94,8 @@ public ViewBuilder withQuery(String dialect, String sql) { } @Override - public ViewBuilder withDefaultCatalog(String defaultCatalog) { - viewVersionBuilder.defaultCatalog(defaultCatalog); + public ViewBuilder withDefaultCatalog(String catalog) { + this.defaultCatalog = catalog; return this; } @@ -152,6 +153,7 @@ private View create(ViewOperations ops) { .versionId(1) .addAllRepresentations(representations) .defaultNamespace(defaultNamespace) + .defaultCatalog(defaultCatalog) .timestampMillis(System.currentTimeMillis()) .putSummary("operation", "create") .build(); @@ -195,6 +197,7 @@ private View replace(ViewOperations ops) { .versionId(maxVersionId + 1) .addAllRepresentations(representations) .defaultNamespace(defaultNamespace) + .defaultCatalog(defaultCatalog) .timestampMillis(System.currentTimeMillis()) .putSummary("operation", "replace") .build(); From 2e0f8a01bc9f69ee42d2a697dee477fed1e81550 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 27 Sep 2023 18:25:22 +0200 Subject: [PATCH 12/14] don't use constants for schema ids --- .../apache/iceberg/view/ViewCatalogTests.java | 33 +++++++------------ 1 file changed, 11 insertions(+), 22 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 4eb2ae8ff10b..a53e10d1d44f 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -42,11 +42,6 @@ import org.junit.jupiter.params.provider.ValueSource; public abstract class ViewCatalogTests { - // the schema ID of SCHEMA / OTHER_SCHEMA are by default set to 0. The schema id of SCHEMA will - // stay 0, but the schema id of OTHER_SCHEMA will be re-assigned to 1 - private static final int EXPECTED_SCHEMA_ID = 0; - private static final int EXPECTED_OTHER_SCHEMA_ID = 1; - protected static final Schema SCHEMA = new Schema( required(3, "id", Types.IntegerType.get(), "unique ID"), @@ -91,9 +86,9 @@ public void basicCreateView() { .first() .extracting(ViewHistoryEntry::versionId) .isEqualTo(1); - assertThat(view.schema().schemaId()).isEqualTo(EXPECTED_SCHEMA_ID); + assertThat(view.schema().schemaId()).isEqualTo(0); assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); - assertThat(view.schemas()).hasSize(1).containsKey(EXPECTED_SCHEMA_ID); + assertThat(view.schemas()).hasSize(1).containsKey(0); assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); assertThat(view.currentVersion()) @@ -101,7 +96,7 @@ public void basicCreateView() { ImmutableViewVersion.builder() .timestampMillis(view.currentVersion().timestampMillis()) .versionId(1) - .schemaId(EXPECTED_SCHEMA_ID) + .schemaId(0) .putSummary("operation", "create") .defaultNamespace(identifier.namespace()) .addRepresentations( @@ -148,9 +143,9 @@ public void completeCreateView() { .first() .extracting(ViewHistoryEntry::versionId) .isEqualTo(1); - assertThat(view.schema().schemaId()).isEqualTo(EXPECTED_SCHEMA_ID); + assertThat(view.schema().schemaId()).isEqualTo(0); assertThat(view.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); - assertThat(view.schemas()).hasSize(1).containsKey(EXPECTED_SCHEMA_ID); + assertThat(view.schemas()).hasSize(1).containsKey(0); assertThat(view.versions()).hasSize(1).containsExactly(view.currentVersion()); assertThat(view.currentVersion()) @@ -158,7 +153,7 @@ public void completeCreateView() { ImmutableViewVersion.builder() .timestampMillis(view.currentVersion().timestampMillis()) .versionId(1) - .schemaId(EXPECTED_SCHEMA_ID) + .schemaId(0) .putSummary("operation", "create") .defaultNamespace(identifier.namespace()) .defaultCatalog(catalog().name()) @@ -824,12 +819,9 @@ public void createOrReplaceView(boolean useCreateOrReplace) { .extracting(ViewHistoryEntry::versionId) .isEqualTo(2); - assertThat(replacedView.schema().schemaId()).isEqualTo(EXPECTED_OTHER_SCHEMA_ID); + assertThat(replacedView.schema().schemaId()).isEqualTo(1); assertThat(replacedView.schema().asStruct()).isEqualTo(OTHER_SCHEMA.asStruct()); - assertThat(replacedView.schemas()) - .hasSize(2) - .containsKey(EXPECTED_SCHEMA_ID) - .containsKey(EXPECTED_OTHER_SCHEMA_ID); + assertThat(replacedView.schemas()).hasSize(2).containsKey(0).containsKey(1); ViewVersion replacedViewVersion = replacedView.currentVersion(); assertThat(replacedView.versions()) @@ -837,7 +829,7 @@ public void createOrReplaceView(boolean useCreateOrReplace) { .containsExactly(viewVersion, replacedViewVersion); assertThat(replacedViewVersion).isNotNull(); assertThat(replacedViewVersion.versionId()).isEqualTo(2); - assertThat(replacedViewVersion.schemaId()).isEqualTo(EXPECTED_OTHER_SCHEMA_ID); + assertThat(replacedViewVersion.schemaId()).isEqualTo(1); assertThat(replacedViewVersion.operation()).isEqualTo("replace"); assertThat(replacedViewVersion.summary()).hasSize(1).containsEntry("operation", "replace"); assertThat(replacedViewVersion.representations()) @@ -1062,10 +1054,7 @@ public void replaceViewVersion() { .element(1) .extracting(ViewHistoryEntry::versionId) .isEqualTo(updatedView.currentVersion().versionId()); - assertThat(updatedView.schemas()) - .hasSize(2) - .containsKey(EXPECTED_SCHEMA_ID) - .containsKey(EXPECTED_OTHER_SCHEMA_ID); + assertThat(updatedView.schemas()).hasSize(2).containsKey(0).containsKey(1); assertThat(updatedView.versions()) .hasSize(2) .containsExactly(viewVersion, updatedView.currentVersion()); @@ -1076,7 +1065,7 @@ public void replaceViewVersion() { assertThat(updatedViewVersion.summary()).hasSize(1).containsEntry("operation", "replace"); assertThat(updatedViewVersion.operation()).isEqualTo("replace"); assertThat(updatedViewVersion.representations()).hasSize(1).containsExactly(trino); - assertThat(updatedViewVersion.schemaId()).isEqualTo(EXPECTED_OTHER_SCHEMA_ID); + assertThat(updatedViewVersion.schemaId()).isEqualTo(1); assertThat(updatedViewVersion.defaultCatalog()).isEqualTo("default"); assertThat(updatedViewVersion.defaultNamespace()).isEqualTo(identifier.namespace()); From d1f587a8fcdfbae595669144f63edda99f9e65c9 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 28 Sep 2023 08:17:30 +0200 Subject: [PATCH 13/14] inline builder --- .../org/apache/iceberg/view/BaseMetastoreViewCatalog.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java index ca37be3eadd3..42eb80a0472a 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java @@ -66,7 +66,6 @@ public ViewBuilder buildView(TableIdentifier identifier) { protected class BaseViewBuilder implements ViewBuilder { private final TableIdentifier identifier; - private final ImmutableViewVersion.Builder viewVersionBuilder = ImmutableViewVersion.builder(); private final Map properties = Maps.newHashMap(); private final List representations = Lists.newArrayList(); private Namespace defaultNamespace = null; @@ -82,7 +81,6 @@ protected BaseViewBuilder(TableIdentifier identifier) { @Override public ViewBuilder withSchema(Schema newSchema) { this.schema = newSchema; - viewVersionBuilder.schemaId(newSchema.schemaId()); return this; } @@ -149,8 +147,9 @@ private View create(ViewOperations ops) { null != defaultNamespace, "Cannot create view without specifying a default namespace"); ViewVersion viewVersion = - viewVersionBuilder + ImmutableViewVersion.builder() .versionId(1) + .schemaId(schema.schemaId()) .addAllRepresentations(representations) .defaultNamespace(defaultNamespace) .defaultCatalog(defaultCatalog) @@ -193,8 +192,9 @@ private View replace(ViewOperations ops) { .orElseGet(metadata::currentVersionId); ViewVersion viewVersion = - viewVersionBuilder + ImmutableViewVersion.builder() .versionId(maxVersionId + 1) + .schemaId(schema.schemaId()) .addAllRepresentations(representations) .defaultNamespace(defaultNamespace) .defaultCatalog(defaultCatalog) From ef8299ac3bb4976a6779624e53514b4a5ad31d6d Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 28 Sep 2023 19:00:21 +0200 Subject: [PATCH 14/14] fixes after rebase --- .../test/java/org/apache/iceberg/view/TestViewMetadata.java | 5 ++--- .../org/apache/iceberg/view/TestViewMetadataParser.java | 4 ++-- .../test/java/org/apache/iceberg/view/ViewCatalogTests.java | 3 ++- .../iceberg/view/ViewMetadataMultipleSQLsForDialect.json | 6 +++--- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java index 15c232e6f92d..7837d9405524 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java @@ -734,11 +734,10 @@ public void viewMetadataWithMultipleSQLForSameDialect() { () -> ViewMetadata.builder() .setLocation("custom-location") - .addSchema( - new Schema(1, Types.NestedField.required(1, "x", Types.LongType.get()))) + .addSchema(new Schema(Types.NestedField.required(1, "x", Types.LongType.get()))) .addVersion( ImmutableViewVersion.builder() - .schemaId(1) + .schemaId(0) .versionId(1) .timestampMillis(23L) .putSummary("operation", "create") diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java index a6ef14cef72f..267195c13350 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java @@ -235,7 +235,7 @@ public void viewMetadataWithMultipleSQLsForDialectShouldBeReadable() throws Exce .versionId(1) .timestampMillis(4353L) .summary(ImmutableMap.of("operation", "create")) - .schemaId(1) + .schemaId(0) .defaultCatalog("some-catalog") .defaultNamespace(Namespace.empty()) .addRepresentations( @@ -291,7 +291,7 @@ public void replaceViewMetadataWithMultipleSQLsForDialect() throws Exception { ViewVersion viewVersion = ImmutableViewVersion.builder() .versionId(2) - .schemaId(1) + .schemaId(0) .timestampMillis(5555L) .summary(ImmutableMap.of("operation", "replace")) .defaultCatalog("some-catalog") diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index a53e10d1d44f..1c95955383d6 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -44,11 +44,12 @@ public abstract class ViewCatalogTests { protected static final Schema SCHEMA = new Schema( + 5, required(3, "id", Types.IntegerType.get(), "unique ID"), required(4, "data", Types.StringType.get())); private static final Schema OTHER_SCHEMA = - new Schema(required(1, "some_id", Types.IntegerType.get())); + new Schema(7, required(1, "some_id", Types.IntegerType.get())); protected abstract C catalog(); diff --git a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json index f849de31d0f1..f5bc04529443 100644 --- a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json +++ b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMultipleSQLsForDialect.json @@ -3,11 +3,11 @@ "format-version": 1, "location": "s3://bucket/test/location", "properties": {"some-key": "some-value"}, - "current-schema-id": 1, + "current-schema-id": 0, "schemas": [ { "type": "struct", - "schema-id": 1, + "schema-id": 0, "fields": [ { "id": 1, @@ -37,7 +37,7 @@ "version-id": 1, "timestamp-ms": 4353, "summary": {"operation":"create"}, - "schema-id": 1, + "schema-id": 0, "default-catalog": "some-catalog", "default-namespace": [], "representations": [