rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528906862



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.nessie;
+
+import com.dremio.nessie.api.TreeApi;
+import com.dremio.nessie.client.NessieClient;
+import com.dremio.nessie.error.BaseNessieClientServerException;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Contents;
+import com.dremio.nessie.model.IcebergTable;
+import com.dremio.nessie.model.ImmutableDelete;
+import com.dremio.nessie.model.ImmutableOperations;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.Operations;
+import com.dremio.nessie.model.Reference;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ *
+ * <p>
+ *   A note on namespaces: Nessie namespaces are implicit and do not need to 
be explicitly created or deleted.
+ *   The create and delete namespace methods are no-ops for the NessieCatalog. 
One can still list namespaces that have
+ *   objects stored in them to assist with namespace-centric catalog 
exploration.
+ * </p>
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements 
AutoCloseable, SupportsNamespaces, Configurable {
+  private static final Logger logger = 
LoggerFactory.getLogger(NessieCatalog.class);
+  private static final Joiner SLASH = Joiner.on("/");
+  private NessieClient client;
+  private String warehouseLocation;
+  private Configuration config;
+  private UpdateableReference reference;
+  private String name;
+  private FileIO fileIO;
+
+  public NessieCatalog() {
+  }
+
+  @Override
+  public void initialize(String inputName, Map<String, String> options) {
+    String fileIOImpl = options.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(config) : 
CatalogUtil.loadFileIO(fileIOImpl, options, config);
+    this.name = inputName == null ? "nessie" : inputName;
+    // remove nessie prefix
+    final Function<String, String> removePrefix = x -> x.replace("nessie.", 
"");
+
+    this.client = NessieClient.withConfig(x -> 
options.get(removePrefix.apply(x)));
+
+    this.warehouseLocation = options.get(CatalogProperties.WAREHOUSE_LOCATION);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter warehouse not set, nessie 
can't store data.");
+    }
+    final String requestedRef = 
options.get(removePrefix.apply(NessieClient.CONF_NESSIE_REF));
+    this.reference = loadReference(requestedRef);
+  }
+
+  @Override
+  public void close() {
+    client.close();
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    TableReference pti = TableReference.parse(tableIdentifier);
+    UpdateableReference newReference = this.reference;
+    if (pti.reference() != null) {
+      newReference = loadReference(pti.reference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.tableIdentifier()), 
newReference, client, fileIO);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().toString(), 
table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    return tableStream(namespace).collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+
+    // We try to drop the table. Simple retry after ref update.
+    boolean threw = true;
+    try {
+      Tasks.foreach(identifier)
+           .retry(5)
+           .stopRetryOn(NessieNotFoundException.class)
+           .throwFailureWhenFinished()
+           .run(this::dropTableInner, BaseNessieClientServerException.class);
+      threw = false;

Review comment:
       Nit: `threw` is no longer needed so this could be simply `return true`. 
That simplifies the logic at the end of the method to just `return false`.
   
   Up to you whether to change this or not. I know some people strongly prefer 
only one exit point from a method.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to