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



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,431 @@
+/*
+ * 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.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Contents;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.EntriesResponse;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Predicate;
+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.TableMetadata;
+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.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+/**
+ * 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 Joiner SLASH = Joiner.on("/");
+  public static final String NESSIE_WAREHOUSE_DIR = "nessie.warehouse.dir";
+  private NessieClient client;
+  private String warehouseLocation;
+  private Configuration config;
+  private UpdateableReference reference;
+  private String name;
+
+  /**
+   * Try to avoid passing parameters via hadoop config. Dynamic catalog 
expects Map instead
+   *
+   * todo replace with #1640 style constructor
+   */
+  public NessieCatalog() {
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String 
url, String warehouseLocation) {
+    this.config = config;
+    this.name = name == null ? "nessie" : name;
+    init(ref, url, warehouseLocation);
+  }
+
+  private void init(String ref, String url, String inputWarehouseLocation) {
+    this.client = NessieClient.withConfig(s -> {
+      if (s.equals(NessieClient.CONF_NESSIE_URL)) {
+        return url == null ? config.get(s) : url;
+      }
+      return config.get(s);
+    });
+
+    this.warehouseLocation = inputWarehouseLocation == null ? 
getWarehouseLocation(config) : inputWarehouseLocation;
+
+    final String requestedRef = ref != null ? ref : 
config.get(NessieClient.CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private static String getWarehouseLocation(Configuration config) {
+    String nessieWarehouseDir = config.get(NESSIE_WAREHOUSE_DIR);
+    if (nessieWarehouseDir != null) {
+      return nessieWarehouseDir;
+    }
+    throw new IllegalStateException("Don't know where to put the nessie 
iceberg data. Please set nessie.warehouse.dir");
+  }
+
+  private UpdateableReference get(String requestedRef) {
+    try {
+      Reference ref = requestedRef == null ? 
client.getTreeApi().getDefaultBranch()
+          : client.getTreeApi().getReferenceByName(requestedRef);
+      return new UpdateableReference(ref, client.getTreeApi());
+    } catch (NessieNotFoundException ex) {
+      if (requestedRef != null) {
+        throw new IllegalArgumentException(String.format("Nessie ref '%s' 
provided via %s does not exist. " +
+          "This ref must exist before creating a NessieCatalog.", 
requestedRef, NessieClient.CONF_NESSIE_REF), ex);
+      }
+
+      throw new IllegalArgumentException(String.format("Nessie does not have 
an existing default branch." +
+        "Either configure an alternative ref via %s or create the default 
branch on the server.",
+          NessieClient.CONF_NESSIE_REF), ex);
+    }
+  }
+
+  @Override
+  public void close() {
+    client.close();
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+
+  private static ContentsKey toKey(TableIdentifier tableIdentifier) {
+    List<String> identifiers = new ArrayList<>();
+    if (tableIdentifier.hasNamespace()) {
+      identifiers.addAll(Arrays.asList(tableIdentifier.namespace().levels()));
+    }
+    identifiers.add(tableIdentifier.name());
+
+    ContentsKey key = new ContentsKey(identifiers);
+    return key;
+  }
+
+  private IcebergTable table(TableIdentifier tableIdentifier) {
+    try {
+      Contents table = 
client.getContentsApi().getContents(toKey(tableIdentifier), 
reference.getHash());
+      if (table instanceof IcebergTable) {
+        return (IcebergTable) table;
+      }
+    } catch (NessieNotFoundException e) {
+      // ignore
+    }
+    return null;
+  }
+
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    ParsedTableIdentifier pti = 
ParsedTableIdentifier.getParsedTableIdentifier(tableIdentifier, new 
HashMap<>());

Review comment:
       How about passing `ImmutableMap.of()` instead of `new HashMap<>()`? That 
avoids unnecessary object creation. Better yet, what about a version of this 
that doesn't need to pass a map if there isn't one?




----------------------------------------------------------------
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