You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/10/12 15:26:00 UTC

[GitHub] [iceberg] rymurr opened a new pull request #1587: DRAFT: Nessie support for core and Spark 2/3

rymurr opened a new pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587


   As per the mailing list [annoucenment](https://lists.apache.org/thread.html/r82ca07231110a410069bc9609b286ae08bce6d4e3256196fb73f2a45%40%3Cdev.iceberg.apache.org%3E 
   ) we would like to contribute integration between Iceberg and Nessie to the Iceberg project.
   
   This PR does the following:
   * adds a `NessieCatalog` for core iceberg acid operations
   * adds nessie support to the catalog and source interfaces for spark 2 and spark 3
   * makes nessie branches and tags addressable for iceberg operations
   
   Please have a look at [Iceberg Spark](https://projectnessie.org/tools/spark/) for a more complete description of Nessie's capabilities with iceberg and [Nessie Features](https://projectnessie.org/features/) for a broader introduction to Nessie.
   
   Note this is currently in draft until a gradle plugin required for testing Nessie has been published.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521388161



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,349 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+                            (tempDir.toURI().toString() + DB_NAME + "/" +
+                             tableName).replace("//",
+                                                "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+                                                               renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+                             .withRecordCount(3)
+                             .withPath(fileLocation)
+                             .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+                             .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    String location2 = table.location().replace("file:", "") + "/data/file2.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location2))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file1 = DataFiles.builder(table.spec())
+                              .withRecordCount(3)
+                              .withPath(location1)
+                              .withFileSizeInBytes(Files.localInput(location2).getLength())
+                              .build();
+
+    DataFile file2 = DataFiles.builder(table.spec())
+                              .withRecordCount(3)
+                              .withPath(location2)
+                              .withFileSizeInBytes(Files.localInput(location1).getLength())
+                              .build();
+
+    // add both data files
+    table.newAppend().appendFile(file1).appendFile(file2).commit();
+
+    // delete file2
+    table.newDelete().deleteFile(file2.path()).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    List<ManifestFile> manifests = table.currentSnapshot().allManifests();
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(location1).exists());
+    Assert.assertTrue(new File(location2).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+    for (ManifestFile manifest : manifests) {
+      Assert.assertTrue(new File(manifest.path().replace("file:", "")).exists());
+    }
+    Assert.assertTrue(new File(
+        ((HasTableOperations) table).operations()
+                                  .current()
+                                  .metadataFileLocation()
+                                  .replace("file:", ""))
+                             .exists());
+  }
+
+  @Test
+  public void testExistingTableUpdate() {
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit();
+
+    icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+
+    // Only 2 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(TABLE_NAME).size());
+    Assert.assertEquals(0, manifestFiles(TABLE_NAME).size());
+    Assert.assertEquals(altered.asStruct(), icebergTable.schema().asStruct());
+
+  }
+
+  @Test(expected = CommitFailedException.class)

Review comment:
       :+1: 
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521391533



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    boolean threw = true;
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+      threw = false;
+    } catch (NessieConflictException ex) {
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } finally {
+      if (threw) {
+        io().deleteFile(newMetadataLocation);
+      }
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }

Review comment:
       thanks for the pointer, copied the Hive Catalog now.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520816362



##########
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);

Review comment:
       Looks like `reference` must never be null, correct?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526543743



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalogBranch.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalogBranch extends BaseTestIceberg {
+
+  public TestCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")

Review comment:
       What does this suppress?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527849789



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalogBranch.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalogBranch extends BaseTestIceberg {
+
+  public TestCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")

Review comment:
       leftover from strict checkstyle checks, removed

##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalogBranch.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalogBranch extends BaseTestIceberg {

Review comment:
       effectively none, removed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520904662



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.ContentsApi;
+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.Branch;
+import com.dremio.nessie.model.Reference;
+import java.io.File;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StructType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public abstract class BaseTestIceberg {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(BaseTestIceberg.class);
+
+  protected static File tempDir;
+  protected NessieCatalog catalog;
+  protected NessieClient client;
+  protected TreeApi tree;
+  protected ContentsApi contents;
+  protected Configuration hadoopConfig;
+  protected final String branch;
+
+  @BeforeClass
+  public static void create() throws Exception {
+    tempDir = java.nio.file.Files.createTempDirectory(
+        "test",
+        PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwxrwx")))
+        .toFile();
+  }
+
+  public BaseTestIceberg(String branch) {
+    this.branch = branch;
+  }
+
+  private void resetData() throws NessieConflictException, NessieNotFoundException {
+    for (Reference r : tree.getAllReferences()) {
+      if (r instanceof Branch) {
+        tree.deleteBranch(r.getName(), r.getHash());
+      } else {
+        tree.deleteTag(r.getName(), r.getHash());
+      }
+    }
+    tree.createReference(Branch.of("main", null));
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    String port = System.getProperty("quarkus.http.test-port", "19120");
+    String path = String.format("http://localhost:%s/api/v1", port);
+    this.client = NessieClient.none(path);
+    tree = client.getTreeApi();
+    contents = client.getContentsApi();
+
+    resetData();
+
+    try {
+      tree.createReference(Branch.of(branch, null));
+    } catch (Exception e) {
+      // ignore, already created. Cant run this in BeforeAll as quarkus hasn't disabled auth
+    }
+
+    hadoopConfig = new Configuration();
+    hadoopConfig.set(NessieClient.CONF_NESSIE_URL, path);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, branch);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_AUTH_TYPE, "NONE");
+    hadoopConfig.set("nessie.warehouse.dir", tempDir.toURI().toString());
+    catalog = initCatalog(branch);
+  }
+
+  NessieCatalog initCatalog(String ref) {
+    NessieCatalog newCatalog = new NessieCatalog();
+    newCatalog.setConf(hadoopConfig);
+    newCatalog.initialize(null, ImmutableMap.of(NessieClient.CONF_NESSIE_REF, ref));
+    return newCatalog;
+  }
+
+  protected Table createTable(TableIdentifier tableIdentifier, int count) {
+    try {
+      return catalog.createTable(tableIdentifier, schema(count));
+    } catch (Throwable t) {
+      LOGGER.error("unable to do create " + tableIdentifier.toString(), t);
+      throw t;
+    }
+  }
+
+  protected void createTable(TableIdentifier tableIdentifier) {
+    Schema schema = new Schema(StructType.of(required(1, "id", LongType.get()))
+                                         .fields());
+    catalog.createTable(tableIdentifier, schema).location();
+  }
+
+  protected static Schema schema(int count) {
+    List<Types.NestedField> fields = new ArrayList<>();
+    for (int i = 0; i < count; i++) {
+      fields.add(required(i, "id" + i, Types.LongType.get()));
+    }
+    return new Schema(Types.StructType.of(fields).fields());
+  }
+
+  void createBranch(String name, String hash) throws NessieNotFoundException, NessieConflictException {
+    if (hash == null) {
+      tree.createReference(Branch.of(name, null));
+    } else {
+      tree.createReference(Branch.of(name, hash));
+    }
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    catalog.close();
+    client.close();
+    catalog = null;
+    client = null;
+    hadoopConfig = null;
+  }
+
+  @AfterClass
+  public static void destroy() throws Exception {
+    tempDir.delete();
+  }
+
+  static String getContent(NessieCatalog catalog, TableIdentifier tableIdentifier) {

Review comment:
       This looks like it should have a more specific name because it returns the metadata location for a table.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526533562



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }

Review comment:
       You can replace this with `Tasks`:
   
   ```java
   Tasks.foreach(identifier)
       .retry(5)
       .stopRetryOn(NessieNotFoundException.class)
       .throwFailureWhenFinished()
       .run(this::dropTableInner)
   ```
   
   `Tasks` is pretty flexible and allows you to configure exponential backoff, whether to retry on all exceptions but a known list (`stopRetryOn`), whether to retry on specific exceptions (`onlyRetryOn`), and set up callbacks (`onFailure`).
   
   It helps us avoid logic like this everywhere.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521616651



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);

Review comment:
       Yeah, I like logging and returning false.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520822572



##########
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);

Review comment:
       Hey @rdblue these parameters are initialised in the `initialize` method (I have moved the method up to near the constructor as it was hidden at teh bottom of the class). The initialisation uses the passed `options` where possible and falls back to `Configuration` if not found. This is to make it compatible with Spark2/3 `IcebergSource`. However I am happy to remove this once `IcebergSource` supports custom catalogs (which I hope to tackle next if its not already being worked on)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520818403



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

Review comment:
       This is fixed now, apologies. I fixed the constructor but didn't remove the comment.
   
   I agree that the Catalog portion of Spark3 should work fine now w/o explicitly adding Nessie (or Glue etc). I believe we still need to update the `IcebergSource` to handle custom (Iceberg) catalogs right?
   
   Is the intention to add the new catalogs to the Iceberg shaded jar? 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520814377



##########
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);

Review comment:
       I don't think that any configuration should come from the Hadoop `Configuration` unless it is used for a Hadoop component, like `HadoopFileIO`. Can you initialize this from the catalog config passed to `initialize`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520815363



##########
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);

Review comment:
       This error message could easily be incorrect because it doesn't use `CONF_NESSIE_REF` directly. It assumes the caller did.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520907870



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,349 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+                            (tempDir.toURI().toString() + DB_NAME + "/" +
+                             tableName).replace("//",
+                                                "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+                                                               renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+                             .withRecordCount(3)
+                             .withPath(fileLocation)
+                             .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+                             .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")

Review comment:
       What is this suppressing?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528914465



##########
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);

Review comment:
       fixed. I was just arguing w/ @jacques-n on this point on Fri ;-) He sided with you.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520847714



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {

Review comment:
       good idea, fixed. Static methods have been moved to a util class and private methods have been moved to the bottom with `Override` methods grouped above




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520824595



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  @Override
+  public void initialize(String inputName, Map<String, String> options) {
+    this.name = inputName;
+    init(options.getOrDefault(NessieClient.CONF_NESSIE_REF, config.get(NessieClient.CONF_NESSIE_REF)),
+         options.getOrDefault(NessieClient.CONF_NESSIE_URL, config.get(NessieClient.CONF_NESSIE_URL)),
+         options.getOrDefault(NESSIE_WAREHOUSE_DIR, config.get(NESSIE_WAREHOUSE_DIR)));
+  }
+
+  public static class Builder {
+    private final Configuration conf;
+    private String url;
+    private String ref;
+    private String warehouseLocation;
+    private String name;
+
+    public Builder(Configuration conf) {
+      this.conf = conf;
+    }
+
+    public Builder setUrl(String url) {
+      this.url = url;
+      return this;
+    }
+
+    public Builder setRef(String ref) {

Review comment:
       This is the default ref, right? Tables can override it. I think that would be a better name if you can use this catalog to load other refs.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528915354



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws IOException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException, IOException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+        (temp.getRoot().toURI().toString() + DB_NAME + "/" +
+            tableName).replace("//",
+            "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+        renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(fileLocation)
+        .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+        .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }

Review comment:
       I think there's something in Flink, but this is an area where we can generally improve tests.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527826541



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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 java.time.Instant;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+public class TableReference {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * Container class to specify a TableIdentifier on a specific Reference or at an Instant in time.
+   */
+  public TableReference(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static TableReference parse(TableIdentifier path) {
+    TableReference pti = parse(path.name());
+    return new TableReference(TableIdentifier.of(path.namespace(), pti.getTableIdentifier().name()),
+        pti.getTimestamp(),
+        pti.getReference());
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static TableReference parse(String path) {
+    // I am assuming tables can't have @ or # symbols
+    if (path.split("@").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one branch in %s", path));
+    }
+    if (path.split("#").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one timestamp in %s", path));
+    }
+
+    if (path.contains("@") && path.contains("#")) {
+      throw new IllegalArgumentException("Invalid table name:" +
+          " # is not allowed (reference by timestamp is not supported)");
+    }
+
+    if (path.contains("@")) {
+      String[] tableRef = path.split("@");
+      TableIdentifier identifier = TableIdentifier.parse(tableRef[0]);
+      return new TableReference(identifier, null, tableRef[1]);
+    }
+
+    if (path.contains("#")) {
+      throw new IllegalArgumentException("Invalid table name:" +
+          " # is not allowed (reference by timestamp is not supported)");
+    }

Review comment:
       Sounds fine to me if you're going to update this soon. I just wanted to make sure we didn't have an unnecessarily long 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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520905063



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.ContentsApi;
+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.Branch;
+import com.dremio.nessie.model.Reference;
+import java.io.File;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StructType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public abstract class BaseTestIceberg {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(BaseTestIceberg.class);
+
+  protected static File tempDir;
+  protected NessieCatalog catalog;
+  protected NessieClient client;
+  protected TreeApi tree;
+  protected ContentsApi contents;
+  protected Configuration hadoopConfig;
+  protected final String branch;
+
+  @BeforeClass
+  public static void create() throws Exception {
+    tempDir = java.nio.file.Files.createTempDirectory(
+        "test",
+        PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwxrwx")))
+        .toFile();
+  }
+
+  public BaseTestIceberg(String branch) {
+    this.branch = branch;
+  }
+
+  private void resetData() throws NessieConflictException, NessieNotFoundException {
+    for (Reference r : tree.getAllReferences()) {
+      if (r instanceof Branch) {
+        tree.deleteBranch(r.getName(), r.getHash());
+      } else {
+        tree.deleteTag(r.getName(), r.getHash());
+      }
+    }
+    tree.createReference(Branch.of("main", null));
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    String port = System.getProperty("quarkus.http.test-port", "19120");
+    String path = String.format("http://localhost:%s/api/v1", port);
+    this.client = NessieClient.none(path);
+    tree = client.getTreeApi();
+    contents = client.getContentsApi();
+
+    resetData();
+
+    try {
+      tree.createReference(Branch.of(branch, null));
+    } catch (Exception e) {
+      // ignore, already created. Cant run this in BeforeAll as quarkus hasn't disabled auth
+    }
+
+    hadoopConfig = new Configuration();
+    hadoopConfig.set(NessieClient.CONF_NESSIE_URL, path);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, branch);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_AUTH_TYPE, "NONE");
+    hadoopConfig.set("nessie.warehouse.dir", tempDir.toURI().toString());
+    catalog = initCatalog(branch);
+  }
+
+  NessieCatalog initCatalog(String ref) {
+    NessieCatalog newCatalog = new NessieCatalog();
+    newCatalog.setConf(hadoopConfig);
+    newCatalog.initialize(null, ImmutableMap.of(NessieClient.CONF_NESSIE_REF, ref));
+    return newCatalog;
+  }
+
+  protected Table createTable(TableIdentifier tableIdentifier, int count) {
+    try {
+      return catalog.createTable(tableIdentifier, schema(count));
+    } catch (Throwable t) {
+      LOGGER.error("unable to do create " + tableIdentifier.toString(), t);
+      throw t;
+    }
+  }
+
+  protected void createTable(TableIdentifier tableIdentifier) {
+    Schema schema = new Schema(StructType.of(required(1, "id", LongType.get()))
+                                         .fields());
+    catalog.createTable(tableIdentifier, schema).location();
+  }
+
+  protected static Schema schema(int count) {
+    List<Types.NestedField> fields = new ArrayList<>();
+    for (int i = 0; i < count; i++) {
+      fields.add(required(i, "id" + i, Types.LongType.get()));
+    }
+    return new Schema(Types.StructType.of(fields).fields());
+  }
+
+  void createBranch(String name, String hash) throws NessieNotFoundException, NessieConflictException {
+    if (hash == null) {
+      tree.createReference(Branch.of(name, null));
+    } else {
+      tree.createReference(Branch.of(name, hash));
+    }

Review comment:
       Looks like this could just always pass `hash`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520849087



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {

Review comment:
       How do you mean? A new interface that no-ops create, load, drop from `SupportsNamespaces`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526531219



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(NessieClient.CONF_NESSIE_REF);

Review comment:
       Similar, it may be easier to configure using just `"ref"`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526536270



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private FileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client,
+      FileIO fileIO) {
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+    this.fileIO = fileIO;
+  }
+
+  @Override
+  protected void doRefresh() {
+    try {
+      reference.refresh();
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to refresh as ref is no longer valid.", e);

Review comment:
       We may want to let the original exception propagate instead of throwing a `RuntimeException` so that people working with the table can take action if the ref is gone. I'm not really sure what the right thing is here.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526541498



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+
+    Table bar = createTable(foobar, 1); // table 1
+    catalog.refresh();

Review comment:
       Isn't it automatically refreshed because it is shared with the `NessieTableOperations`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520819942



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");

Review comment:
       Can't this refresh and complete the operation?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520822354



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {

Review comment:
       Util methods seem to be mixed in. I think it may help readability if these were at the bottom, or were static methods in a `NessieUtil` class.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520825564



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private String applicationId() {
+    String appId = null;
+    TableMetadata current = current();
+    if (current != null) {
+      Snapshot snapshot = current.currentSnapshot();
+      if (snapshot != null) {
+        Map<String, String> summary = snapshot.summary();
+        appId = summary.get("spark.app.id");

Review comment:
       If an update doesn't create a snapshot, then this method will return the app ID that committed the last snapshot. That may not be correct. Should we create a class in core to hold this information instead? Then we could set it somewhere in Spark and Flink so you'd always have identifiers without needing to resort to reflection?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520855525



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();

Review comment:
       Cool, that makes sense to me. Have removed the comment. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r529579391



##########
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);

Review comment:
       @rdblue just checking the checkstyle.xml in the project and it has a regexp of `(?i)log(ger)?\.(debug|info|warn|error)\(.*` for the logger. eg LOG, log, logger, LOGGER are all acceptable. Want me to switch it to just allow `LOGGER`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521615564



##########
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);

Review comment:
       We support the Hive warehouse property because that's how to set it up in Hive. Since we are introducing new configuration for Nessie, I'd really rather not make it so people can depend on using the Hadoop config. Then we will never be able to get rid of it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: DRAFT: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r503394937



##########
File path: spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceNessieTables.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.spark.source;
+
+import com.dremio.nessie.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.nessie.NessieCatalog;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class TestIcebergSourceNessieTables extends TestIcebergSourceTablesBase {
+
+  private static TableIdentifier currentIdentifier;
+
+  private NessieClient client;
+  private String branch;
+
+  private Configuration getConfig() throws IOException {

Review comment:
       The nessie specific tests all modify spark settings and reset the settings at the end. This is to interfere as little as possible w/ the 'normal' iceberg path.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526098717



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();

Review comment:
       Yes and no, but the option should at least be there. However I think doing this properly is a bigger question. Whether to refresh depends on the isolation level and the use case. 
   
   Currently if two tables were modified by a different process on the same branch and an iceberg client, who had both those tables cached, committed to only one then the two tables would be out of sync on that iceberg client. But I think that would be true if they were using Hive or Hadoop also, correct? I think rather than addressing this use case in here it would be better to have a wider discussion on how to handle consistency across iceberg catalogs. Thoughts?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511181909



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java
##########
@@ -50,6 +50,7 @@ public TestCreateTableAsSelect(String catalogName, String implementation, Map<St
   @After
   public void removeTables() {
     sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);

Review comment:
       Why was this needed?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527756373



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");

Review comment:
       agreed, fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521377749



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.ContentsApi;
+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.Branch;
+import com.dremio.nessie.model.Reference;
+import java.io.File;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StructType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public abstract class BaseTestIceberg {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(BaseTestIceberg.class);
+
+  protected static File tempDir;
+  protected NessieCatalog catalog;
+  protected NessieClient client;
+  protected TreeApi tree;
+  protected ContentsApi contents;
+  protected Configuration hadoopConfig;
+  protected final String branch;
+
+  @BeforeClass
+  public static void create() throws Exception {
+    tempDir = java.nio.file.Files.createTempDirectory(
+        "test",
+        PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwxrwx")))
+        .toFile();
+  }
+
+  public BaseTestIceberg(String branch) {
+    this.branch = branch;
+  }
+
+  private void resetData() throws NessieConflictException, NessieNotFoundException {
+    for (Reference r : tree.getAllReferences()) {
+      if (r instanceof Branch) {
+        tree.deleteBranch(r.getName(), r.getHash());
+      } else {
+        tree.deleteTag(r.getName(), r.getHash());
+      }
+    }
+    tree.createReference(Branch.of("main", null));
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    String port = System.getProperty("quarkus.http.test-port", "19120");
+    String path = String.format("http://localhost:%s/api/v1", port);
+    this.client = NessieClient.none(path);
+    tree = client.getTreeApi();
+    contents = client.getContentsApi();
+
+    resetData();
+
+    try {
+      tree.createReference(Branch.of(branch, null));
+    } catch (Exception e) {
+      // ignore, already created. Cant run this in BeforeAll as quarkus hasn't disabled auth
+    }
+
+    hadoopConfig = new Configuration();
+    hadoopConfig.set(NessieClient.CONF_NESSIE_URL, path);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, branch);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_AUTH_TYPE, "NONE");
+    hadoopConfig.set("nessie.warehouse.dir", tempDir.toURI().toString());
+    catalog = initCatalog(branch);
+  }
+
+  NessieCatalog initCatalog(String ref) {
+    NessieCatalog newCatalog = new NessieCatalog();
+    newCatalog.setConf(hadoopConfig);
+    newCatalog.initialize(null, ImmutableMap.of(NessieClient.CONF_NESSIE_REF, ref));
+    return newCatalog;
+  }
+
+  protected Table createTable(TableIdentifier tableIdentifier, int count) {
+    try {
+      return catalog.createTable(tableIdentifier, schema(count));
+    } catch (Throwable t) {
+      LOGGER.error("unable to do create " + tableIdentifier.toString(), t);
+      throw t;
+    }
+  }
+
+  protected void createTable(TableIdentifier tableIdentifier) {
+    Schema schema = new Schema(StructType.of(required(1, "id", LongType.get()))
+                                         .fields());
+    catalog.createTable(tableIdentifier, schema).location();
+  }
+
+  protected static Schema schema(int count) {
+    List<Types.NestedField> fields = new ArrayList<>();
+    for (int i = 0; i < count; i++) {
+      fields.add(required(i, "id" + i, Types.LongType.get()));
+    }
+    return new Schema(Types.StructType.of(fields).fields());
+  }
+
+  void createBranch(String name, String hash) throws NessieNotFoundException, NessieConflictException {
+    if (hash == null) {
+      tree.createReference(Branch.of(name, null));
+    } else {
+      tree.createReference(Branch.of(name, hash));
+    }
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    catalog.close();
+    client.close();
+    catalog = null;
+    client = null;
+    hadoopConfig = null;
+  }
+
+  @AfterClass
+  public static void destroy() throws Exception {
+    tempDir.delete();
+  }
+
+  static String getContent(NessieCatalog catalog, TableIdentifier tableIdentifier) {

Review comment:
       agreed! Bizarre that wasn't used in the first place :-)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521618396



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();

Review comment:
       @rymurr, do we need to request a refresh for all other tables that use this ref?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511176255



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+
+    this.client = NessieClient.withConfig(s -> {
+      if (s.equals(NessieClient.CONF_NESSIE_URL)) {
+        return url == null ? config.get(s) : url;
+      }
+      return config.get(s);
+    });
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(NessieClient.CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");
+    if (nessieWarehouseDir != null) {
+      return nessieWarehouseDir;
+    }
+    String hiveWarehouseDir = config.get("hive.metastore.warehouse.dir");
+    if (hiveWarehouseDir != null) {
+      return hiveWarehouseDir;
+    }
+    String defaultFS = config.get("fs.defaultFS");
+    if (defaultFS != null) {
+      return defaultFS + "/" + ICEBERG_HADOOP_WAREHOUSE_BASE;
+    }
+    throw new IllegalStateException("Don't know where to put the nessie iceberg data. " +
+        "Please set one of the following:\n" +
+        "nessie.warehouse.dir\n" +
+        "hive.metastore.warehouse.dir\n" +
+        "fs.defaultFS.");
+  }
+
+  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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier)
+          .collect(Collectors.toList());

Review comment:
       Looks like this will return all tables underneath the given namespace, even if they are nested in other namespaces?
   
   I haven't tested this in spark, does it work as expected?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527709821



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    }
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  public TreeApi getTreeApi() {

Review comment:
       :thinking: I expect there was a reason at some point. It is only used by tests so I have made it package private for now. The original intention was likely to expose CRUD ops on branches/tags to the user but I think a more structured interface would be better for that




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527720861



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private FileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client,
+      FileIO fileIO) {
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+    this.fileIO = fileIO;
+  }
+
+  @Override
+  protected void doRefresh() {
+    try {
+      reference.refresh();
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to refresh as ref is no longer valid.", e);
+    }
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    boolean threw = true;
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+      threw = false;
+    } catch (NessieConflictException ex) {
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);

Review comment:
       I think my comment above covers this as well. The only sensible way this could ever throw is if someone deleted your branch between when you loaded the table and when you tried to commit.
   
   I have made that a bit more explicit in the comment




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520816050



##########
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);

Review comment:
       Here as well, I don't think this should pull config from `Configuration`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r519779322



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private static String applicationId() {

Review comment:
       Ahh, thanks for that. Much cleaner this way. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526543532



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalogBranch.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalogBranch extends BaseTestIceberg {
+
+  public TestCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+    TableIdentifier foobaz = TableIdentifier.of("foo", "baz");
+    Table bar = createTable(foobar, 1); // table 1
+    createTable(foobaz, 1); // table 2
+    catalog.refresh();
+    createBranch("test", catalog.currentHash());
+
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "test");
+
+    NessieCatalog newCatalog = initCatalog("test");
+    String initialMetadataLocation = metadataLocation(newCatalog, foobar);
+    Assert.assertEquals(initialMetadataLocation, metadataLocation(catalog, foobar));
+    Assert.assertEquals(metadataLocation(newCatalog, foobaz), metadataLocation(catalog, foobaz));

Review comment:
       Context helpers would make this test more readable and would be helpful if it ever fails.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526068733



##########
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);

Review comment:
       Ok, removed. The consequence of this is the custom catalog work for `IcebergSource` has to be done before the next release if we want a valid/usable nessie in the release




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527756502



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+
+    Table bar = createTable(foobar, 1); // table 1

Review comment:
       :+1: 
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511176395



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+
+    this.client = NessieClient.withConfig(s -> {
+      if (s.equals(NessieClient.CONF_NESSIE_URL)) {
+        return url == null ? config.get(s) : url;
+      }
+      return config.get(s);
+    });
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(NessieClient.CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");
+    if (nessieWarehouseDir != null) {
+      return nessieWarehouseDir;
+    }
+    String hiveWarehouseDir = config.get("hive.metastore.warehouse.dir");
+    if (hiveWarehouseDir != null) {
+      return hiveWarehouseDir;
+    }
+    String defaultFS = config.get("fs.defaultFS");
+    if (defaultFS != null) {
+      return defaultFS + "/" + ICEBERG_HADOOP_WAREHOUSE_BASE;
+    }
+    throw new IllegalStateException("Don't know where to put the nessie iceberg data. " +
+        "Please set one of the following:\n" +
+        "nessie.warehouse.dir\n" +
+        "hive.metastore.warehouse.dir\n" +
+        "fs.defaultFS.");
+  }
+
+  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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier)
+          .collect(Collectors.toList());
+    } catch (NessieNotFoundException ex) {
+      throw new RuntimeException("Unable to list tables due to missing ref.", ex);

Review comment:
       Probably shouldn't use `RuntimeException` here. How about `NoSuchNamespaceException`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520903750



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.ContentsApi;
+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.Branch;
+import com.dremio.nessie.model.Reference;
+import java.io.File;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StructType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public abstract class BaseTestIceberg {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(BaseTestIceberg.class);
+
+  protected static File tempDir;
+  protected NessieCatalog catalog;
+  protected NessieClient client;
+  protected TreeApi tree;
+  protected ContentsApi contents;
+  protected Configuration hadoopConfig;
+  protected final String branch;
+
+  @BeforeClass
+  public static void create() throws Exception {
+    tempDir = java.nio.file.Files.createTempDirectory(
+        "test",
+        PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwxrwx")))

Review comment:
       Most tests use `@Rule TemporaryFolder temp` so that JUnit handles temp lifecycle. I'd recommend doing that here, too.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511172248



##########
File path: build.gradle
##########
@@ -891,6 +924,49 @@ project(':iceberg-pig') {
   }
 }
 
+project(':iceberg-nessie') {
+  apply plugin: 'org.projectnessie'

Review comment:
       What's happening in the Nessie plugin?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-716499739


   > Thanks, @rymurr! This looks like a great start. I commented in a few places where I noticed some things. Overall, you're going in the right direction.
   > 
   > How do you want to start getting this in? I think it would be good to break it up a bit into smaller commits with a few tests. That way, we can iterate more quickly and we reduce the amount of scope that reviewers need to keep track of. Would it be possible to add just the Nessie module with a few tests and then move on to updating Spark modules?
   
   Thanks a lot for the feedback @rdblue I will rework this PR to be just the nessie module and will open another for Spark. Will follow the pattern from #1640 for the spark PR.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527755370



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+
+    Table bar = createTable(foobar, 1); // table 1
+    catalog.refresh();

Review comment:
       References aren't refreshed on commits. So while the ref is shared its out of date everywhere.
   
   This normally isn't a problem but since in this test we are creating a branch off the current state of the 'test-branch-hash' branch we explicitly need the up to date hash. Note: in the `TableOperations` the metadata would be up to date so querying the data would work fine. In normal practice the ref being out of date will only mean we don't get any updates from other users (not the recommended usage pattern) or we would get a merge conflict if we updated the table after someone else has.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527885857



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws IOException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException, IOException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+        (temp.getRoot().toURI().toString() + DB_NAME + "/" +
+            tableName).replace("//",
+            "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+        renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(fileLocation)
+        .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+        .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }

Review comment:
       Can do, I original copied this from one of the catalog tests. I believe those are all package private static methods. Do you know of any test class that has these utils available publicly?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526535107



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    }
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  public TreeApi getTreeApi() {

Review comment:
       Why is this `public`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526538697



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private FileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client,
+      FileIO fileIO) {
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+    this.fileIO = fileIO;
+  }
+
+  @Override
+  protected void doRefresh() {
+    try {
+      reference.refresh();
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to refresh as ref is no longer valid.", e);
+    }
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    boolean threw = true;
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+      threw = false;
+    } catch (NessieConflictException ex) {
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);

Review comment:
       `CommitFailedException` is used to trigger a table refresh and a retry. Throwing it for `NessieConflictException` seems correct to me, but reading the error messages makes me less sure.
   
   If the ref is a tag, then we don't want to retry because it can't be updated, right? In that case, this should throw some other exception because the table is read-only.
   
   If the ref is a branch, then the ref should be refreshed using the normal retry logic so everything looks good. `doRefresh` will update the ref and then update the table content.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: DRAFT: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r503395596



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
##########
@@ -133,16 +135,29 @@ protected Table findTable(DataSourceOptions options, Configuration conf) {
     Optional<String> path = options.get("path");
     Preconditions.checkArgument(path.isPresent(), "Cannot open table: path is not set");
 
-    if (path.get().contains("/")) {
-      HadoopTables tables = new HadoopTables(conf);
-      return tables.load(path.get());
+    if (nessie(options.asMap(), conf)) {

Review comment:
       We identify Nessie as the core catalog/source when there are specific parameters available on the classpath or hadoop config. The idea here is to be fully backwards compatible w/ Hive and Hadoop catalogs.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520899130



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/ParsedTableIdentifier.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.client.NessieClient;
+import java.time.Instant;
+import java.util.Map;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+public class ParsedTableIdentifier {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * container class to hold all options in a Nessie table name.
+   */
+  public ParsedTableIdentifier(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(TableIdentifier path) {
+    return getParsedTableIdentifier(path, ImmutableMap.of());
+  }
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(String path, Map<String, String> properties) {
+    // I am assuming tables can't have @ or # symbols
+    if (path.split("@").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one branch in %s", path));
+    }
+    if (path.split("#").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one timestamp in %s", path));
+    }
+
+    if (path.contains("@") && path.contains("#")) {
+      throw new IllegalArgumentException("Currently we don't support referencing by timestamp, # is not allowed in " +
+          "the table name");
+    }
+
+    if (path.contains("@")) {
+      String[] tableRef = path.split("@");
+      TableIdentifier identifier = TableIdentifier.parse(tableRef[0]);
+      return new ParsedTableIdentifier(identifier, null, tableRef[1]);
+    }
+
+    if (path.contains("#")) {
+      throw new IllegalArgumentException("Currently we don't support referencing by timestamp, # is not allowed in " +

Review comment:
       We try to use simpler error messages and avoid referring to specific people, like "we" or "you". A good rule of thumb is "Cannot [some action]: [problem[ [(suggestion to fix)]" or "Invalid [something]: [problem]". How about "Invalid table name: # is not allowed (reference by timestamp is not supported)"?

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/ParsedTableIdentifier.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.client.NessieClient;
+import java.time.Instant;
+import java.util.Map;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+public class ParsedTableIdentifier {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * container class to hold all options in a Nessie table name.
+   */
+  public ParsedTableIdentifier(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(TableIdentifier path) {
+    return getParsedTableIdentifier(path, ImmutableMap.of());
+  }
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(String path, Map<String, String> properties) {
+    // I am assuming tables can't have @ or # symbols
+    if (path.split("@").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one branch in %s", path));
+    }
+    if (path.split("#").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one timestamp in %s", path));
+    }
+
+    if (path.contains("@") && path.contains("#")) {
+      throw new IllegalArgumentException("Currently we don't support referencing by timestamp, # is not allowed in " +
+          "the table name");
+    }
+
+    if (path.contains("@")) {
+      String[] tableRef = path.split("@");
+      TableIdentifier identifier = TableIdentifier.parse(tableRef[0]);
+      return new ParsedTableIdentifier(identifier, null, tableRef[1]);
+    }
+
+    if (path.contains("#")) {
+      throw new IllegalArgumentException("Currently we don't support referencing by timestamp, # is not allowed in " +
+          "the table name");
+    }
+
+    TableIdentifier identifier = TableIdentifier.parse(path);
+    String reference = properties.get(NessieClient.CONF_NESSIE_REF);
+    return new ParsedTableIdentifier(identifier, null, reference);
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(TableIdentifier path, Map<String, String> properties) {

Review comment:
       Can we use a simpler verb, like `parse` or `parseTableIdentifier`? It's wordy to use "get" and then a past tense verb.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-713805766


   Opening this up as a reviewable PR to get early feedback. 


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526545279



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws IOException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException, IOException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+        (temp.getRoot().toURI().toString() + DB_NAME + "/" +
+            tableName).replace("//",
+            "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+        renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(fileLocation)
+        .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+        .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }

Review comment:
       Could this be refactored to be smaller and use a couple of util functions? Something like `addFile(table, filename)` could help.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528914919



##########
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:
       fixed. I like your way better too..just a hangover from the refactor




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r529579391



##########
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);

Review comment:
       @rdblue just checking the checkstyle,xml in the project and it has a regexp of `(?i)log(ger)?\.(debug|info|warn|error)\(.*` for the logger. eg LOG, log, logger, LOGGER are all acceptable. Want me to switch it to just allow `LOGGER`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526544477



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestDefaultCatalogBranch.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * test tag operations with a default tag set by server.
+ */
+public class TestDefaultCatalogBranch extends BaseTestIceberg {
+
+  public TestDefaultCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+    TableIdentifier foobaz = TableIdentifier.of("foo", "baz");
+    createTable(foobar, 1); // table 1
+    createTable(foobaz, 1); // table 2
+
+    catalog.refresh();
+    tree.createReference(Branch.of("FORWARD", catalog.currentHash()));
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "FORWARD");
+    NessieCatalog forwardCatalog = initCatalog("FORWARD");
+    forwardCatalog.loadTable(foobaz).updateSchema().addColumn("id1", Types.LongType.get()).commit();
+    forwardCatalog.loadTable(foobar).updateSchema().addColumn("id1", Types.LongType.get()).commit();
+    Assert.assertNotEquals(metadataLocation(forwardCatalog, foobar),
+                               metadataLocation(catalog, foobar));
+    Assert.assertNotEquals(metadataLocation(forwardCatalog, foobaz),
+                               metadataLocation(catalog, foobaz));
+
+    System.out.println(metadataLocation(forwardCatalog, foobar));
+    System.out.println(metadataLocation(catalog, foobar));
+
+    forwardCatalog.refresh();
+    tree.assignBranch("main",
+        tree.getReferenceByName("main").getHash(),
+        Branch.of("main", forwardCatalog.currentHash()));
+
+    catalog.refresh();
+
+    System.out.println(metadataLocation(forwardCatalog, foobar));
+    System.out.println(metadataLocation(catalog, foobar));
+
+    Assert.assertEquals(metadataLocation(forwardCatalog, foobar),
+                            metadataLocation(catalog, foobar));
+    Assert.assertEquals(metadataLocation(forwardCatalog, foobaz),
+                            metadataLocation(catalog, foobaz));
+
+    catalog.dropTable(foobar);
+    catalog.dropTable(foobaz);
+    tree.deleteBranch("FORWARD", tree.getReferenceByName("FORWARD").getHash());

Review comment:
       This is another test that doesn't seem very different from the others. Can these be combined into a single suite with good test case names that calls out what is unique about each test?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: DRAFT: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r503397068



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java
##########
@@ -56,6 +56,8 @@ public void cleanNamespaces() {
 
   @Test
   public void testCreateNamespace() {
+    // Nessie namespaces are explicit and do not need to be explicitly managed
+    Assume.assumeFalse(catalogName.endsWith("testnessie"));

Review comment:
       The concept of a namespace is implicit in Nessie and are therefore not managed through the normal `SupportsNamespaces` interface. We skip tests of this interface when the catalog is a `NessieCatalog`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520822774



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {

Review comment:
       Is there a more specific name for this? It isn't clear what `catalog.getHash()` should be.
   
   Also, style nit: we avoid using `get` where a more specific verb would add value.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526536690



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    }
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() throws NessieNotFoundException {
+    reference.refresh();
+  }
+
+  public String currentHash() {
+    return reference.getHash();
+  }
+
+  private IcebergTable table(TableIdentifier tableIdentifier) {
+    try {
+      Contents table = client.getContentsApi().getContents(NessieUtil.toKey(tableIdentifier), reference.getHash());
+      if (table instanceof IcebergTable) {

Review comment:
       `NessieTableOperations` calls `unwrap` instead of using `instanceof`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521614936



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

Review comment:
       > Is the intention to add the new catalogs to the Iceberg shaded jar?
   
   I think it depends. If a catalog pulls in a ton of dependencies and requires updating a lot of the shaded Jar's documentation, then it comes at a high cost. On the other hand, if it uses existing bundled libraries or libraries that can be pulled from the Spark runtime, then it would be easier.
   
   > I believe we still need to update the IcebergSource to handle custom (Iceberg) catalogs right?
   
   Yes, we will need to come up with a way for IcebergSource to work with custom catalogs. Spark has a way for the source to return a catalog and identifier that is used instead of the source directly. That's a much better model, but the problem is that we don't necessarily know what the catalog should be. And, if we redirect to a catalog, we will need to also have a catalog that can load Hadoop tables from a URI. I think this is more of a follow-up.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528914002



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalogBranch.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalogBranch extends BaseTestIceberg {
+
+  public TestCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+    TableIdentifier foobaz = TableIdentifier.of("foo", "baz");
+    Table bar = createTable(foobar, 1); // table 1
+    createTable(foobaz, 1); // table 2
+    catalog.refresh();
+    createBranch("test", catalog.currentHash());
+
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "test");
+
+    NessieCatalog newCatalog = initCatalog("test");
+    String initialMetadataLocation = metadataLocation(newCatalog, foobar);
+    Assert.assertEquals(initialMetadataLocation, metadataLocation(catalog, foobar));
+    Assert.assertEquals(metadataLocation(newCatalog, foobaz), metadataLocation(catalog, foobaz));

Review comment:
       Looks a lot better!




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527676926



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(NessieClient.CONF_NESSIE_REF);

Review comment:
       :+1:




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527708348



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);

Review comment:
       If `to` table has already been dropped and our ref wasn't up to date we would get a conflict exception. This is similar to a merge conflict in git: the table has been changed by someone else so there is a conflict exception. The fact that the change is a delete is irrelevant nessie, same error would be thrown if the table had been appended to or the schema were to be changed.
   
   I have opened projectnessie/nessie#477 to track the lack of documentation on the exception classes. In the meantime I have added a comment here




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-724950249


   Thanks a lot for the review @rdblue, I think have addressed all of your comments. I think further discussion on namespace interfaces and commit metadata is warranted.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511179065



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      this.table = null;
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieNotFoundException | NessieConflictException ex) {

Review comment:
       Is this right for `NotFoundException`? Iceberg will retry failed commits.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511181217



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
##########
@@ -133,16 +135,29 @@ protected Table findTable(DataSourceOptions options, Configuration conf) {
     Optional<String> path = options.get("path");
     Preconditions.checkArgument(path.isPresent(), "Cannot open table: path is not set");
 
-    if (path.get().contains("/")) {
-      HadoopTables tables = new HadoopTables(conf);
-      return tables.load(path.get());
+    if (nessie(options.asMap(), conf)) {

Review comment:
       This is probably an area to revisit. Right now, this is written to have minimal changes between 2.4.x and 3.0.x, but I think we will probably want to route all loading from here through a catalog. That will allow us to delegate all of this to Nessie or Hive the same way.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527919539



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws IOException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException, IOException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+        (temp.getRoot().toURI().toString() + DB_NAME + "/" +
+            tableName).replace("//",
+            "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+        renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(fileLocation)
+        .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+        .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    String location2 = table.location().replace("file:", "") + "/data/file2.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location2))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file1 = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(location1)
+        .withFileSizeInBytes(Files.localInput(location2).getLength())
+        .build();
+
+    DataFile file2 = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(location2)
+        .withFileSizeInBytes(Files.localInput(location1).getLength())
+        .build();
+
+    // add both data files
+    table.newAppend().appendFile(file1).appendFile(file2).commit();
+
+    // delete file2
+    table.newDelete().deleteFile(file2.path()).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    List<ManifestFile> manifests = table.currentSnapshot().allManifests();
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(location1).exists());
+    Assert.assertTrue(new File(location2).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+    for (ManifestFile manifest : manifests) {
+      Assert.assertTrue(new File(manifest.path().replace("file:", "")).exists());
+    }
+    Assert.assertTrue(new File(
+        ((HasTableOperations) table).operations()
+            .current()
+            .metadataFileLocation()
+            .replace("file:", ""))
+        .exists());
+  }
+
+  @Test
+  public void testExistingTableUpdate() {
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit();
+
+    icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+
+    // Only 2 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(TABLE_NAME).size());
+    Assert.assertEquals(0, manifestFiles(TABLE_NAME).size());
+    Assert.assertEquals(altered.asStruct(), icebergTable.schema().asStruct());
+
+  }
+
+  @Test
+  public void testFailure() throws NessieNotFoundException, NessieConflictException {
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    Branch branch = (Branch) client.getTreeApi().getReferenceByName(BRANCH);
+
+    IcebergTable table = client.getContentsApi().getContents(KEY, BRANCH).unwrap(IcebergTable.class).get();
+
+    client.getContentsApi().setContents(KEY, branch.getName(), branch.getHash(), "",
+        IcebergTable.of("dummytable.metadata.json"));
+
+    AssertHelpers.assertThrows("Update schema fails with conflict exception, ref not up to date",
+        CommitFailedException.class,
+        () -> icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit());

Review comment:
       Cleaned this up a bit more. We do both tests you suggested, thought it may not be clear its happening. Let me know if the extra comments aren't enough.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527850047



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestDefaultCatalogBranch.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * test tag operations with a default tag set by server.
+ */
+public class TestDefaultCatalogBranch extends BaseTestIceberg {
+
+  public TestDefaultCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+    TableIdentifier foobaz = TableIdentifier.of("foo", "baz");
+    createTable(foobar, 1); // table 1
+    createTable(foobaz, 1); // table 2
+
+    catalog.refresh();
+    tree.createReference(Branch.of("FORWARD", catalog.currentHash()));
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "FORWARD");
+    NessieCatalog forwardCatalog = initCatalog("FORWARD");
+    forwardCatalog.loadTable(foobaz).updateSchema().addColumn("id1", Types.LongType.get()).commit();
+    forwardCatalog.loadTable(foobar).updateSchema().addColumn("id1", Types.LongType.get()).commit();
+    Assert.assertNotEquals(metadataLocation(forwardCatalog, foobar),
+                               metadataLocation(catalog, foobar));
+    Assert.assertNotEquals(metadataLocation(forwardCatalog, foobaz),
+                               metadataLocation(catalog, foobaz));
+
+    System.out.println(metadataLocation(forwardCatalog, foobar));
+    System.out.println(metadataLocation(catalog, foobar));

Review comment:
       removed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520909846



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/ParsedTableIdentifier.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.client.NessieClient;
+import java.time.Instant;
+import java.util.Map;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+public class ParsedTableIdentifier {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * container class to hold all options in a Nessie table name.
+   */
+  public ParsedTableIdentifier(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(TableIdentifier path) {
+    return getParsedTableIdentifier(path, ImmutableMap.of());
+  }
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(String path, Map<String, String> properties) {

Review comment:
       Does this need properties? The only thing that is used is an optional ref. That could be passed by itself rather than as a map.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526539000



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private FileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client,
+      FileIO fileIO) {
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+    this.fileIO = fileIO;
+  }
+
+  @Override
+  protected void doRefresh() {
+    try {
+      reference.refresh();
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to refresh as ref is no longer valid.", e);
+    }
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    boolean threw = true;
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+      threw = false;
+    } catch (NessieConflictException ex) {
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);

Review comment:
       Like above, do we know this is the ref because the current ref already loaded the table?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-715618075


   Thanks, @rymurr! This looks like a great start. I commented in a few places where I noticed some things. Overall, you're going in the right direction.
   
   How do you want to start getting this in? I think it would be good to break it up a bit into smaller commits with a few tests. That way, we can iterate more quickly and we reduce the amount of scope that reviewers need to keep track of. Would it be possible to add just the Nessie module with a few tests and then move on to updating Spark modules?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520829341



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());

Review comment:
       agreed, fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520826489



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);

Review comment:
       What about doing this delete in a `finally` block if `threw` is false? That's usually a better way than catching `Throwable` and wrapping it in a `RuntimeException`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527714837



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    }
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() throws NessieNotFoundException {
+    reference.refresh();
+  }
+
+  public String currentHash() {
+    return reference.getHash();
+  }
+
+  private IcebergTable table(TableIdentifier tableIdentifier) {
+    try {
+      Contents table = client.getContentsApi().getContents(NessieUtil.toKey(tableIdentifier), reference.getHash());
+      if (table instanceof IcebergTable) {

Review comment:
       nice catch




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526069072



##########
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);

Review comment:
       removed later as stated above




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526542496



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+
+    Table bar = createTable(foobar, 1); // table 1

Review comment:
       Similar, it would be good to name this `table` for clarity later on in this long test 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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520848457



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {

Review comment:
       changed to `currentHash`. Thoughts? The method returns the current hash as teh catalog understands it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526544282



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestDefaultCatalogBranch.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * test tag operations with a default tag set by server.
+ */
+public class TestDefaultCatalogBranch extends BaseTestIceberg {
+
+  public TestDefaultCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+    TableIdentifier foobaz = TableIdentifier.of("foo", "baz");
+    createTable(foobar, 1); // table 1
+    createTable(foobaz, 1); // table 2
+
+    catalog.refresh();
+    tree.createReference(Branch.of("FORWARD", catalog.currentHash()));
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "FORWARD");
+    NessieCatalog forwardCatalog = initCatalog("FORWARD");
+    forwardCatalog.loadTable(foobaz).updateSchema().addColumn("id1", Types.LongType.get()).commit();
+    forwardCatalog.loadTable(foobar).updateSchema().addColumn("id1", Types.LongType.get()).commit();
+    Assert.assertNotEquals(metadataLocation(forwardCatalog, foobar),
+                               metadataLocation(catalog, foobar));
+    Assert.assertNotEquals(metadataLocation(forwardCatalog, foobaz),
+                               metadataLocation(catalog, foobaz));
+
+    System.out.println(metadataLocation(forwardCatalog, foobar));
+    System.out.println(metadataLocation(catalog, foobar));

Review comment:
       Nit: println usage.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511181610



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/SparkCatalog.java
##########
@@ -103,6 +106,10 @@ protected Catalog buildIcebergCatalog(String name, CaseInsensitiveStringMap opti
         String warehouseLocation = options.get("warehouse");
         return new HadoopCatalog(name, conf, warehouseLocation);
 
+      case "nessie":
+        String defaultBranch = options.getOrDefault("nessie_ref", "main");
+        String nessieUrl = options.get("nessie_url");
+        return new NessieCatalog(name, conf, defaultBranch, nessieUrl);

Review comment:
       Please have a look at #1640, I'd like to standardize how we do this. I do like using `type = nessie`, so we may want to have a lookup that points to the `NessieCatalog` implementation.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511172817



##########
File path: build.gradle
##########
@@ -693,6 +709,8 @@ if (jdkVersion == '8') {
       // Vectorized reads need more memory
       maxHeapSize '2500m'
     }
+    // start and stop quarkus for nessie tests
+    tasks.test.dependsOn("quarkus-start").finalizedBy("quarkus-stop")

Review comment:
       From the comments in the Iceberg sync, it sounds like this is running a stand-alone Nessie server? Is that something we could handle like the current Hive MetaStore tests, where each test suite creates a new metastore and tears it down after the suite runs?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526535107



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    }
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  public TreeApi getTreeApi() {

Review comment:
       Why is this `public`?
   
   Not a blocker, just normal code lint.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528910205



##########
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));

Review comment:
       Did you intend to change this to `"ref"`? Your reply seemed to imply that: https://github.com/apache/iceberg/pull/1587#discussion_r526531219




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: DRAFT: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r503393453



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.client.NessieClient.AuthType;
+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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  public static final String CONF_NESSIE_URL = "nessie.url";
+  public static final String CONF_NESSIE_USERNAME = "nessie.username";
+  public static final String CONF_NESSIE_PASSWORD = "nessie.password";
+  public static final String CONF_NESSIE_AUTH_TYPE = "nessie.auth_type";
+  public static final String NESSIE_AUTH_TYPE_DEFAULT = "BASIC";
+  public static final String CONF_NESSIE_REF = "nessie.ref";
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+    String path = url == null ? config.get(CONF_NESSIE_URL) : url;
+    String username = config.get(CONF_NESSIE_USERNAME);
+    String password = config.get(CONF_NESSIE_PASSWORD);
+    String authTypeStr = config.get(CONF_NESSIE_AUTH_TYPE, NESSIE_AUTH_TYPE_DEFAULT);
+    AuthType authType = AuthType.valueOf(authTypeStr);
+    this.client = new NessieClient(authType, path, username, password);
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");

Review comment:
       not sure if this is the best way to get hold of a directory to write tables into. Anyone have any suggestions?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520854219



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);

Review comment:
       agreed, fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527697410



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");

Review comment:
       yup. Not sure why I was being lazy. Fixed now




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526534794



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);

Review comment:
       This can't happen if the `to` table has already been dropped? Seems like this assumes that the `NessieNotFoundException` refers to the ref.
   
   Maybe we're guaranteed that the ref hasn't changed because this isn't a `NessieConflictException`? If so, a comment would help.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520906352



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalog.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 java.util.List;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalog extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-catalog-branch";
+
+  public TestCatalog() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void test() {
+    createTable(TableIdentifier.of("foo", "bar"));
+    List<TableIdentifier> tables = catalog.listTables(Namespace.of("foo"));
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("bar", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.renameTable(TableIdentifier.of("foo", "bar"), TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(null);
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("baz", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.dropTable(TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(Namespace.empty());
+    Assert.assertTrue(tables.isEmpty());

Review comment:
       I prefer to separate tests into distinct cases. It looks like this combines `testRename` with `testListTables`. Combining test cases into a single method causes failures to prevent other tests from running, which makes the whole suite harder to work with.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520908518



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,349 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+                            (tempDir.toURI().toString() + DB_NAME + "/" +
+                             tableName).replace("//",
+                                                "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+                                                               renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+                             .withRecordCount(3)
+                             .withPath(fileLocation)
+                             .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+                             .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    String location2 = table.location().replace("file:", "") + "/data/file2.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location2))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file1 = DataFiles.builder(table.spec())
+                              .withRecordCount(3)
+                              .withPath(location1)
+                              .withFileSizeInBytes(Files.localInput(location2).getLength())
+                              .build();
+
+    DataFile file2 = DataFiles.builder(table.spec())
+                              .withRecordCount(3)
+                              .withPath(location2)
+                              .withFileSizeInBytes(Files.localInput(location1).getLength())
+                              .build();
+
+    // add both data files
+    table.newAppend().appendFile(file1).appendFile(file2).commit();
+
+    // delete file2
+    table.newDelete().deleteFile(file2.path()).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    List<ManifestFile> manifests = table.currentSnapshot().allManifests();
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(location1).exists());
+    Assert.assertTrue(new File(location2).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+    for (ManifestFile manifest : manifests) {
+      Assert.assertTrue(new File(manifest.path().replace("file:", "")).exists());
+    }
+    Assert.assertTrue(new File(
+        ((HasTableOperations) table).operations()
+                                  .current()
+                                  .metadataFileLocation()
+                                  .replace("file:", ""))
+                             .exists());
+  }
+
+  @Test
+  public void testExistingTableUpdate() {
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit();
+
+    icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+
+    // Only 2 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(TABLE_NAME).size());
+    Assert.assertEquals(0, manifestFiles(TABLE_NAME).size());
+    Assert.assertEquals(altered.asStruct(), icebergTable.schema().asStruct());
+
+  }
+
+  @Test(expected = CommitFailedException.class)

Review comment:
       Can you use `AssertHelpers.assertThrows` instead? That way, you can add assertions after the exception to check other things, like that the table has not been modified.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511175494



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.client.NessieClient.AuthType;
+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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  public static final String CONF_NESSIE_URL = "nessie.url";
+  public static final String CONF_NESSIE_USERNAME = "nessie.username";
+  public static final String CONF_NESSIE_PASSWORD = "nessie.password";
+  public static final String CONF_NESSIE_AUTH_TYPE = "nessie.auth_type";
+  public static final String NESSIE_AUTH_TYPE_DEFAULT = "BASIC";
+  public static final String CONF_NESSIE_REF = "nessie.ref";
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+    String path = url == null ? config.get(CONF_NESSIE_URL) : url;
+    String username = config.get(CONF_NESSIE_USERNAME);
+    String password = config.get(CONF_NESSIE_PASSWORD);
+    String authTypeStr = config.get(CONF_NESSIE_AUTH_TYPE, NESSIE_AUTH_TYPE_DEFAULT);
+    AuthType authType = AuthType.valueOf(authTypeStr);
+    this.client = new NessieClient(authType, path, username, password);
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");

Review comment:
       In general, I would discourage depending so heavily on Hadoop `Configuration`. Spark and Flink have a way to pass catalog-specific options, which is the best way to configure catalogs.
   
   There is some discussion about this in #1640. I think that catalogs should primarily depend on config passed in a string map, and should only use Hadoop `Configuration` when dependencies (like `HadoopFileIO` or `HiveClient`) require it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526075735



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private String applicationId() {
+    String appId = null;
+    TableMetadata current = current();
+    if (current != null) {
+      Snapshot snapshot = current.currentSnapshot();
+      if (snapshot != null) {
+        Map<String, String> summary = snapshot.summary();
+        appId = summary.get("spark.app.id");

Review comment:
       cool, we can add it to the `CaseInsensitiveStrngMap` when `SparkCatalog` is initialised? I am not sure of the effect of changing this map from inside `SparkCatalog` though, perhaps passing a copy w/ the extra params?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520901378



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {

Review comment:
       Just an interface that omits those. All this needs is to list namespaces, not do anything else.
   
   I guess we can take a closer look if anything else needs this.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520836248



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);

Review comment:
       You are correct `NessieNotFoundException`  refers to the `ref`. If the table were deleted it would be a conflict exception. This is similar to comparing the error modes of git if you committed to a non-existent ref compared to a merge conflict in the case of changing files in the repo




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520820327



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.

Review comment:
       In this case, just remove the purge. We do that in our catalog as well because we never delete data as a result of a user action. We garbage collect it later.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527760314



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+
+    Table bar = createTable(foobar, 1); // table 1
+    catalog.refresh();
+    createBranch("test", catalog.currentHash());
+
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "test");
+
+    NessieCatalog newCatalog = initCatalog("test");
+    String initialMetadataLocation = metadataLocation(newCatalog, foobar);
+    Assert.assertEquals(initialMetadataLocation, metadataLocation(catalog, foobar));
+
+    bar.updateSchema().addColumn("id1", Types.LongType.get()).commit();
+
+    // metadata location changed no longer matches
+    Assert.assertNotEquals(metadataLocation(catalog, foobar), metadataLocation(newCatalog, foobar));
+
+    // points to the previous metadata location
+    Assert.assertEquals(initialMetadataLocation, metadataLocation(newCatalog, foobar));
+
+
+    String mainHash = tree.getReferenceByName(BRANCH).getHash();
+    // catalog created with ref and no hash points to same catalog as above
+    NessieCatalog refCatalog = initCatalog("test");
+    Assert.assertEquals(metadataLocation(newCatalog, foobar), metadataLocation(refCatalog, foobar));
+    // catalog created with ref and hash points to
+    NessieCatalog refHashCatalog = initCatalog(mainHash);
+    Assert.assertEquals(metadataLocation(catalog, foobar), metadataLocation(refHashCatalog, foobar));
+
+    // asking for table@branch gives expected regardless of catalog
+    Assert.assertEquals(metadataLocation(newCatalog, foobar),
+        metadataLocation(catalog, TableIdentifier.of("foo", "bar@test")));
+    // asking for table@branch#hash gives expected regardless of catalog
+    Assert.assertEquals(metadataLocation(catalog, foobar),
+        metadataLocation(catalog, TableIdentifier.of("foo", "bar@" + mainHash)));

Review comment:
       :+1:




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-725523056


   Thanks again @rdblue for the patient code review, hopefully this next round looks better. I will be offline for a few days so will address any comments when I return. Unless @jacques-n or @laurentgo get there first :-)


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521386680



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,349 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+                            (tempDir.toURI().toString() + DB_NAME + "/" +
+                             tableName).replace("//",
+                                                "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+                                                               renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+                             .withRecordCount(3)
+                             .withPath(fileLocation)
+                             .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+                             .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")

Review comment:
       absolutely nothing! It got carried over from the nessie repo (which has militant checkstyle rules) and I forgot to remove it. Gone now.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526298059



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

Review comment:
       Which mechanism were you thinking for this? Was it `LookupCatalog`? That is rather scala-y but it works. I have something basic working along the lines of `LookupCatalog` but I have some concerns about it. Shall I post a PR as a straw man?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520906352



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalog.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 java.util.List;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalog extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-catalog-branch";
+
+  public TestCatalog() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void test() {
+    createTable(TableIdentifier.of("foo", "bar"));
+    List<TableIdentifier> tables = catalog.listTables(Namespace.of("foo"));
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("bar", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.renameTable(TableIdentifier.of("foo", "bar"), TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(null);
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("baz", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.dropTable(TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(Namespace.empty());
+    Assert.assertTrue(tables.isEmpty());

Review comment:
       I prefer to separate tests into distinct cases. It looks like this combines `testRename` with `testListTables`. Combining test cases into a single method causes failures to prevent other tests from running, which makes the whole suite harder to work with.
   
   It is also a lot easier to spot missing test cases and add new ones when tests are separate. I'd recommend taking a look at most of these test suites.
   
   That said, I think that you'll be the primary reviewers here so in the end it is mostly up to you what conventions you want to maintain.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520901378



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {

Review comment:
       Just an interface that omits those. All this needs is to list namespaces, not do anything else.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526540334



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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 java.time.Instant;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+public class TableReference {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * Container class to specify a TableIdentifier on a specific Reference or at an Instant in time.
+   */
+  public TableReference(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static TableReference parse(TableIdentifier path) {
+    TableReference pti = parse(path.name());
+    return new TableReference(TableIdentifier.of(path.namespace(), pti.getTableIdentifier().name()),
+        pti.getTimestamp(),
+        pti.getReference());
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static TableReference parse(String path) {
+    // I am assuming tables can't have @ or # symbols
+    if (path.split("@").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one branch in %s", path));
+    }
+    if (path.split("#").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one timestamp in %s", path));
+    }
+
+    if (path.contains("@") && path.contains("#")) {
+      throw new IllegalArgumentException("Invalid table name:" +
+          " # is not allowed (reference by timestamp is not supported)");
+    }
+
+    if (path.contains("@")) {
+      String[] tableRef = path.split("@");
+      TableIdentifier identifier = TableIdentifier.parse(tableRef[0]);
+      return new TableReference(identifier, null, tableRef[1]);
+    }
+
+    if (path.contains("#")) {
+      throw new IllegalArgumentException("Invalid table name:" +
+          " # is not allowed (reference by timestamp is not supported)");
+    }

Review comment:
       Looks like this check is the only one needed for `#`. It doesn't matter if the identifier also contains `@` and it also doesn't matter if there is more than 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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520852398



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private String applicationId() {
+    String appId = null;
+    TableMetadata current = current();
+    if (current != null) {
+      Snapshot snapshot = current.currentSnapshot();
+      if (snapshot != null) {
+        Map<String, String> summary = snapshot.summary();
+        appId = summary.get("spark.app.id");

Review comment:
       Ahh, good point. Do you mean setting some static state somewhere which holds the current app id? I don't love setting static state, is there a way to tell if the snapshot is the 'correct' snapshot we are looking for?
   
   We were just talking today about how to better handle Nessie commit info, perhaps somethign we could discuss tomorrow on the sync call? cc @jacques-n




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520837887



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.

Review comment:
       fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520820455



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.

Review comment:
       What is this referring to?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
nastra commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r644141910



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private String applicationId() {
+    String appId = null;
+    TableMetadata current = current();
+    if (current != null) {
+      Snapshot snapshot = current.currentSnapshot();
+      if (snapshot != null) {
+        Map<String, String> summary = snapshot.summary();
+        appId = summary.get("spark.app.id");

Review comment:
       I opened https://github.com/apache/iceberg/pull/2664 to address that




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527823831



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);

Review comment:
       Yeah, I meant using `CatalogProperties.WAREHOUSE_LOCATION`. That's what the other catalogs use for a config property.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520910207



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,349 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+                            (tempDir.toURI().toString() + DB_NAME + "/" +
+                             tableName).replace("//",
+                                                "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+                                                               renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+                             .withRecordCount(3)
+                             .withPath(fileLocation)
+                             .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+                             .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    String location2 = table.location().replace("file:", "") + "/data/file2.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location2))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {

Review comment:
       We typically use a continuation indent of 2 indents / 4 spaces, rather than aligning with the previous method call.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520833770



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  @Override
+  public void initialize(String inputName, Map<String, String> options) {
+    this.name = inputName;
+    init(options.getOrDefault(NessieClient.CONF_NESSIE_REF, config.get(NessieClient.CONF_NESSIE_REF)),
+         options.getOrDefault(NessieClient.CONF_NESSIE_URL, config.get(NessieClient.CONF_NESSIE_URL)),
+         options.getOrDefault(NESSIE_WAREHOUSE_DIR, config.get(NESSIE_WAREHOUSE_DIR)));
+  }
+
+  public static class Builder {
+    private final Configuration conf;
+    private String url;
+    private String ref;
+    private String warehouseLocation;
+    private String name;
+
+    public Builder(Configuration conf) {
+      this.conf = conf;
+    }
+
+    public Builder setUrl(String url) {
+      this.url = url;
+      return this;
+    }
+
+    public Builder setRef(String ref) {

Review comment:
       I've deleted the builder as its superfluous after #1640




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511182190



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java
##########
@@ -56,6 +56,8 @@ public void cleanNamespaces() {
 
   @Test
   public void testCreateNamespace() {
+    // Nessie namespaces are explicit and do not need to be explicitly managed
+    Assume.assumeFalse(catalogName.endsWith("testnessie"));

Review comment:
       There are a lot of tests that need this. Should we separate the test cases into different suites?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521366373



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/ParsedTableIdentifier.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.client.NessieClient;
+import java.time.Instant;
+import java.util.Map;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+public class ParsedTableIdentifier {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * container class to hold all options in a Nessie table name.
+   */
+  public ParsedTableIdentifier(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(TableIdentifier path) {
+    return getParsedTableIdentifier(path, ImmutableMap.of());
+  }
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(String path, Map<String, String> properties) {
+    // I am assuming tables can't have @ or # symbols
+    if (path.split("@").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one branch in %s", path));
+    }
+    if (path.split("#").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one timestamp in %s", path));
+    }
+
+    if (path.contains("@") && path.contains("#")) {
+      throw new IllegalArgumentException("Currently we don't support referencing by timestamp, # is not allowed in " +
+          "the table name");
+    }
+
+    if (path.contains("@")) {
+      String[] tableRef = path.split("@");
+      TableIdentifier identifier = TableIdentifier.parse(tableRef[0]);
+      return new ParsedTableIdentifier(identifier, null, tableRef[1]);
+    }
+
+    if (path.contains("#")) {
+      throw new IllegalArgumentException("Currently we don't support referencing by timestamp, # is not allowed in " +
+          "the table name");
+    }
+
+    TableIdentifier identifier = TableIdentifier.parse(path);
+    String reference = properties.get(NessieClient.CONF_NESSIE_REF);
+    return new ParsedTableIdentifier(identifier, null, reference);
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(TableIdentifier path, Map<String, String> properties) {

Review comment:
       fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520823736



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  @Override
+  public void initialize(String inputName, Map<String, String> options) {

Review comment:
       Ignore my comments above, since it looks like you've already added this. Can you merge this with `init` and the constructors?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520829057



##########
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);

Review comment:
       correct. The only way to return from `get` is with a valid reference, otherwise an exception would be thrown. Would you prefer an explicit null check here?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527679913



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }

Review comment:
       :+1:




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526542290



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");

Review comment:
       Using the table name "foo.bar" is fine, but using `foobar` as a variable name makes the test harder to read because it isn't obvious what `metadataLocation(catalog, foobar)` does exactly. If foobar is a `Namespace`, then it would be the default location for a table, for example. It would be easier to read if this were something more descriptive, like `tableIdent`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526546123



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws IOException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException, IOException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+        (temp.getRoot().toURI().toString() + DB_NAME + "/" +
+            tableName).replace("//",
+            "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+        renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(fileLocation)
+        .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+        .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    String location2 = table.location().replace("file:", "") + "/data/file2.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location2))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file1 = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(location1)
+        .withFileSizeInBytes(Files.localInput(location2).getLength())
+        .build();
+
+    DataFile file2 = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(location2)
+        .withFileSizeInBytes(Files.localInput(location1).getLength())
+        .build();
+
+    // add both data files
+    table.newAppend().appendFile(file1).appendFile(file2).commit();
+
+    // delete file2
+    table.newDelete().deleteFile(file2.path()).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    List<ManifestFile> manifests = table.currentSnapshot().allManifests();
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(location1).exists());
+    Assert.assertTrue(new File(location2).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+    for (ManifestFile manifest : manifests) {
+      Assert.assertTrue(new File(manifest.path().replace("file:", "")).exists());
+    }
+    Assert.assertTrue(new File(
+        ((HasTableOperations) table).operations()
+            .current()
+            .metadataFileLocation()
+            .replace("file:", ""))
+        .exists());
+  }
+
+  @Test
+  public void testExistingTableUpdate() {
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit();
+
+    icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+
+    // Only 2 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(TABLE_NAME).size());
+    Assert.assertEquals(0, manifestFiles(TABLE_NAME).size());
+    Assert.assertEquals(altered.asStruct(), icebergTable.schema().asStruct());
+
+  }
+
+  @Test
+  public void testFailure() throws NessieNotFoundException, NessieConflictException {
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    Branch branch = (Branch) client.getTreeApi().getReferenceByName(BRANCH);
+
+    IcebergTable table = client.getContentsApi().getContents(KEY, BRANCH).unwrap(IcebergTable.class).get();
+
+    client.getContentsApi().setContents(KEY, branch.getName(), branch.getHash(), "",
+        IcebergTable.of("dummytable.metadata.json"));
+
+    AssertHelpers.assertThrows("Update schema fails with conflict exception, ref not up to date",
+        CommitFailedException.class,
+        () -> icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit());

Review comment:
       Do you have a test for concurrent writes to the same table from multiple catalogs? That would be good with two cases: two tables that share the same ref (loaded by the same catalog) and two tables that were loaded by different catalogs and have separate refs.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527884954



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws IOException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException, IOException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+        (temp.getRoot().toURI().toString() + DB_NAME + "/" +
+            tableName).replace("//",
+            "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+        renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(fileLocation)
+        .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+        .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    String location2 = table.location().replace("file:", "") + "/data/file2.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location2))
+        .schema(schema)
+        .named("test")
+        .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file1 = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(location1)
+        .withFileSizeInBytes(Files.localInput(location2).getLength())
+        .build();
+
+    DataFile file2 = DataFiles.builder(table.spec())
+        .withRecordCount(3)
+        .withPath(location2)
+        .withFileSizeInBytes(Files.localInput(location1).getLength())
+        .build();
+
+    // add both data files
+    table.newAppend().appendFile(file1).appendFile(file2).commit();
+
+    // delete file2
+    table.newDelete().deleteFile(file2.path()).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    List<ManifestFile> manifests = table.currentSnapshot().allManifests();
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(location1).exists());
+    Assert.assertTrue(new File(location2).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+    for (ManifestFile manifest : manifests) {
+      Assert.assertTrue(new File(manifest.path().replace("file:", "")).exists());
+    }
+    Assert.assertTrue(new File(
+        ((HasTableOperations) table).operations()
+            .current()
+            .metadataFileLocation()
+            .replace("file:", ""))
+        .exists());
+  }
+
+  @Test
+  public void testExistingTableUpdate() {
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit();
+
+    icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+
+    // Only 2 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(TABLE_NAME).size());
+    Assert.assertEquals(0, manifestFiles(TABLE_NAME).size());
+    Assert.assertEquals(altered.asStruct(), icebergTable.schema().asStruct());
+
+  }
+
+  @Test
+  public void testFailure() throws NessieNotFoundException, NessieConflictException {
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    Branch branch = (Branch) client.getTreeApi().getReferenceByName(BRANCH);
+
+    IcebergTable table = client.getContentsApi().getContents(KEY, BRANCH).unwrap(IcebergTable.class).get();
+
+    client.getContentsApi().setContents(KEY, branch.getName(), branch.getHash(), "",
+        IcebergTable.of("dummytable.metadata.json"));
+
+    AssertHelpers.assertThrows("Update schema fails with conflict exception, ref not up to date",
+        CommitFailedException.class,
+        () -> icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit());

Review comment:
       `TestCatalogBranch` does this for the positive case (ie concurrent changes that succeed) and internal nessie for teh negative case. I have added a negative case to `TestCatalogBranch` also. (and renamed the class)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527721641



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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 java.time.Instant;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+public class TableReference {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * Container class to specify a TableIdentifier on a specific Reference or at an Instant in time.
+   */
+  public TableReference(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {

Review comment:
       fixed. The pythonista in me doesn't like `get` either :-)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521377949



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.ContentsApi;
+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.Branch;
+import com.dremio.nessie.model.Reference;
+import java.io.File;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StructType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public abstract class BaseTestIceberg {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(BaseTestIceberg.class);
+
+  protected static File tempDir;
+  protected NessieCatalog catalog;
+  protected NessieClient client;
+  protected TreeApi tree;
+  protected ContentsApi contents;
+  protected Configuration hadoopConfig;
+  protected final String branch;
+
+  @BeforeClass
+  public static void create() throws Exception {
+    tempDir = java.nio.file.Files.createTempDirectory(
+        "test",
+        PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwxrwx")))

Review comment:
       agreed! Bizarre that wasn't used in the first place :-)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521362452



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/ParsedTableIdentifier.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.client.NessieClient;
+import java.time.Instant;
+import java.util.Map;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+public class ParsedTableIdentifier {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * container class to hold all options in a Nessie table name.
+   */
+  public ParsedTableIdentifier(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(TableIdentifier path) {
+    return getParsedTableIdentifier(path, ImmutableMap.of());
+  }
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(String path, Map<String, String> properties) {
+    // I am assuming tables can't have @ or # symbols
+    if (path.split("@").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one branch in %s", path));
+    }
+    if (path.split("#").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one timestamp in %s", path));
+    }
+
+    if (path.contains("@") && path.contains("#")) {
+      throw new IllegalArgumentException("Currently we don't support referencing by timestamp, # is not allowed in " +
+          "the table name");
+    }
+
+    if (path.contains("@")) {
+      String[] tableRef = path.split("@");
+      TableIdentifier identifier = TableIdentifier.parse(tableRef[0]);
+      return new ParsedTableIdentifier(identifier, null, tableRef[1]);
+    }
+
+    if (path.contains("#")) {
+      throw new IllegalArgumentException("Currently we don't support referencing by timestamp, # is not allowed in " +

Review comment:
       agreed, fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526534372



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");

Review comment:
       I'd prefer an error message with more context, like `Failed to rename X to Y`. There's no guarantee that the error message from Nessie has that information. It probably has information about the state of the ref, rather than what was being attempted.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511178920



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      this.table = null;
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),

Review comment:
       Doesn't look like the format here is quite correct. Missing a space?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520830542



##########
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:
       agreed, fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520817143



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());

Review comment:
       I think it would be helpful for `get` to have a better name for uses like this. What about `findReference` or `loadReference`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521385726



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalog.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 java.util.List;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalog extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-catalog-branch";
+
+  public TestCatalog() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void test() {
+    createTable(TableIdentifier.of("foo", "bar"));
+    List<TableIdentifier> tables = catalog.listTables(Namespace.of("foo"));
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("bar", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.renameTable(TableIdentifier.of("foo", "bar"), TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(null);
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("baz", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.dropTable(TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(Namespace.empty());
+    Assert.assertTrue(tables.isEmpty());

Review comment:
       I prefer splitting as well however in this case the state is important. I have split into 3 tests and introduced a lexicographical ordering annotation. Its a little more cluttered this way but let me know what you think, happy to go either way.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511176395



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+
+    this.client = NessieClient.withConfig(s -> {
+      if (s.equals(NessieClient.CONF_NESSIE_URL)) {
+        return url == null ? config.get(s) : url;
+      }
+      return config.get(s);
+    });
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(NessieClient.CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");
+    if (nessieWarehouseDir != null) {
+      return nessieWarehouseDir;
+    }
+    String hiveWarehouseDir = config.get("hive.metastore.warehouse.dir");
+    if (hiveWarehouseDir != null) {
+      return hiveWarehouseDir;
+    }
+    String defaultFS = config.get("fs.defaultFS");
+    if (defaultFS != null) {
+      return defaultFS + "/" + ICEBERG_HADOOP_WAREHOUSE_BASE;
+    }
+    throw new IllegalStateException("Don't know where to put the nessie iceberg data. " +
+        "Please set one of the following:\n" +
+        "nessie.warehouse.dir\n" +
+        "hive.metastore.warehouse.dir\n" +
+        "fs.defaultFS.");
+  }
+
+  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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier)
+          .collect(Collectors.toList());
+    } catch (NessieNotFoundException ex) {
+      throw new RuntimeException("Unable to list tables due to missing ref.", ex);

Review comment:
       Probably shouldn't use `RuntimeException` here. How about `NotFoundException`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520849682



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  @Override
+  public void initialize(String inputName, Map<String, String> options) {
+    this.name = inputName;
+    init(options.getOrDefault(NessieClient.CONF_NESSIE_REF, config.get(NessieClient.CONF_NESSIE_REF)),
+         options.getOrDefault(NessieClient.CONF_NESSIE_URL, config.get(NessieClient.CONF_NESSIE_URL)),
+         options.getOrDefault(NESSIE_WAREHOUSE_DIR, config.get(NESSIE_WAREHOUSE_DIR)));
+  }
+
+  public static class Builder {
+    private final Configuration conf;
+    private String url;
+    private String ref;
+    private String warehouseLocation;
+    private String name;
+
+    public Builder(Configuration conf) {
+      this.conf = conf;
+    }
+
+    public Builder setUrl(String url) {

Review comment:
       removed the builder and any URL refs I found




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526529343



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;

Review comment:
       It seems strange to me to default this in the catalog rather than in the tests. I would probably use a precondition to validate it isn't null instead.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: DRAFT: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r503395922



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java
##########
@@ -83,12 +90,40 @@ public static void dropWarehouse() {
   protected final SupportsNamespaces validationNamespaceCatalog;
   protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table");
   protected final String tableName;
+  protected NessieClient client;
+  protected String branch;
 
   public SparkCatalogTestBase(String catalogName, String implementation, Map<String, String> config) {
     this.catalogName = catalogName;
-    this.validationCatalog = catalogName.equals("testhadoop") ?
-        new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse) :
-        catalog;
+    switch (catalogName) {
+      case "testhadoop":
+        this.validationCatalog = new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse);
+        break;
+      case "testnessie":
+        String path = "http://localhost:19121/api/v1";
+        branch = config.get("nessie_ref");
+        setHadoopConfig(path, branch);
+
+        this.client = new NessieClient(NessieClient.AuthType.NONE, path, null, null);
+        try {
+          try {
+            this.client.getTreeApi().createEmptyBranch(branch);

Review comment:
       All Nessie tests are run in their own branch to not interfere with parallel test execution




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521378924



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.ContentsApi;
+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.Branch;
+import com.dremio.nessie.model.Reference;
+import java.io.File;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StructType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public abstract class BaseTestIceberg {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(BaseTestIceberg.class);
+
+  protected static File tempDir;
+  protected NessieCatalog catalog;
+  protected NessieClient client;
+  protected TreeApi tree;
+  protected ContentsApi contents;
+  protected Configuration hadoopConfig;
+  protected final String branch;
+
+  @BeforeClass
+  public static void create() throws Exception {
+    tempDir = java.nio.file.Files.createTempDirectory(
+        "test",
+        PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwxrwx")))
+        .toFile();
+  }
+
+  public BaseTestIceberg(String branch) {
+    this.branch = branch;
+  }
+
+  private void resetData() throws NessieConflictException, NessieNotFoundException {
+    for (Reference r : tree.getAllReferences()) {
+      if (r instanceof Branch) {
+        tree.deleteBranch(r.getName(), r.getHash());
+      } else {
+        tree.deleteTag(r.getName(), r.getHash());
+      }
+    }
+    tree.createReference(Branch.of("main", null));
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    String port = System.getProperty("quarkus.http.test-port", "19120");
+    String path = String.format("http://localhost:%s/api/v1", port);
+    this.client = NessieClient.none(path);
+    tree = client.getTreeApi();
+    contents = client.getContentsApi();
+
+    resetData();
+
+    try {
+      tree.createReference(Branch.of(branch, null));
+    } catch (Exception e) {
+      // ignore, already created. Cant run this in BeforeAll as quarkus hasn't disabled auth
+    }
+
+    hadoopConfig = new Configuration();
+    hadoopConfig.set(NessieClient.CONF_NESSIE_URL, path);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, branch);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_AUTH_TYPE, "NONE");
+    hadoopConfig.set("nessie.warehouse.dir", tempDir.toURI().toString());
+    catalog = initCatalog(branch);
+  }
+
+  NessieCatalog initCatalog(String ref) {
+    NessieCatalog newCatalog = new NessieCatalog();
+    newCatalog.setConf(hadoopConfig);
+    newCatalog.initialize(null, ImmutableMap.of(NessieClient.CONF_NESSIE_REF, ref));
+    return newCatalog;
+  }
+
+  protected Table createTable(TableIdentifier tableIdentifier, int count) {
+    try {
+      return catalog.createTable(tableIdentifier, schema(count));
+    } catch (Throwable t) {
+      LOGGER.error("unable to do create " + tableIdentifier.toString(), t);
+      throw t;
+    }
+  }
+
+  protected void createTable(TableIdentifier tableIdentifier) {
+    Schema schema = new Schema(StructType.of(required(1, "id", LongType.get()))
+                                         .fields());
+    catalog.createTable(tableIdentifier, schema).location();
+  }
+
+  protected static Schema schema(int count) {
+    List<Types.NestedField> fields = new ArrayList<>();
+    for (int i = 0; i < count; i++) {
+      fields.add(required(i, "id" + i, Types.LongType.get()));
+    }
+    return new Schema(Types.StructType.of(fields).fields());
+  }
+
+  void createBranch(String name, String hash) throws NessieNotFoundException, NessieConflictException {
+    if (hash == null) {
+      tree.createReference(Branch.of(name, null));
+    } else {
+      tree.createReference(Branch.of(name, hash));
+    }
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    catalog.close();
+    client.close();
+    catalog = null;
+    client = null;
+    hadoopConfig = null;
+  }
+
+  @AfterClass
+  public static void destroy() throws Exception {
+    tempDir.delete();
+  }
+
+  static String getContent(NessieCatalog catalog, TableIdentifier tableIdentifier) {

Review comment:
       fixed, contents is an internal nessie name for the objects stored in the Nessie object database. Changed it to content held for iceberg




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527671572



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;

Review comment:
       Just to make sure I am looking at the right line of the diff: you mean the name right? I have updated the test to set the catalog name to `nessie` but I was following the other Catalog impls which set the name to eg 'hive' or 'hadoop' if name is unset.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528905901



##########
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);

Review comment:
       Nit: static final constants should use upper case names, like LOGGER. I'm not sure why style checks didn't catch this.
   
   (Not a blocker)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520834377



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  @Override
+  public void initialize(String inputName, Map<String, String> options) {

Review comment:
       done :-)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520901997



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private String applicationId() {
+    String appId = null;
+    TableMetadata current = current();
+    if (current != null) {
+      Snapshot snapshot = current.currentSnapshot();
+      if (snapshot != null) {
+        Map<String, String> summary = snapshot.summary();
+        appId = summary.get("spark.app.id");

Review comment:
       Maybe we should pass this in through catalog properties?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520827966



##########
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);

Review comment:
       re-jigged this and above a little bit to make it clear that the hadoop config is only used as a fallback. Hopefully that is more clear. As stated before I hope to remove `Configuration` for anything but IO in a further PR.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527675321



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);

Review comment:
       I am not sure I understand what you mean by 'warehouse' here. The original version of this patch used the same as the hadoop catalog (`conf.get("fs.defaultFS") + "/" + ICEBERG_HADOOP_WAREHOUSE_BASE`) but I believe a change was suggested/requested. I personally would much prefer a standard approach than the custom nessie directory. Do you prefer using the Hadoop style one above or the `CatalogProperties.WAREHOUSE_LOCATION` one? 
   
   I have switched in the most recent patch to using `WAREHOUSE_LOCATION`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520902708



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/UpdateableReference.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.Hash;
+import com.dremio.nessie.model.Reference;
+import org.apache.iceberg.exceptions.NotFoundException;
+
+class UpdateableReference {
+
+  private Reference reference;
+  private final TreeApi client;
+
+  UpdateableReference(Reference reference, TreeApi client) {
+    this.reference = reference;
+    this.client = client;
+  }
+
+  public boolean refresh() {
+    if (reference instanceof Hash) {
+      return false;
+    }
+    Reference oldReference = reference;
+    try {
+      reference = client.getReferenceByName(reference.getName());
+    } catch (NessieNotFoundException e) {
+      throw new NotFoundException(e, "Failure refreshing data, table no longer exists.");

Review comment:
       Table no longer exists? Or the ref no longer exists?
   
   Also, `NotFoundException` is for files that don't exist. Tables should use `NoSuchTableException`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-731360511


   Thanks again for the thorough review @rdblue I have updated w/ your suggestions and rebased. hope I didn't miss anything!


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528938903



##########
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));

Review comment:
       I missed the `removePrefix` call. Thanks!




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526530130



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);

Review comment:
       Other catalogs use "warehouse" instead of a catalog-specific property. It would be slightly better for consistency to do the same, although it is fine this way. If the `NessieClient` is expecting config like this, it may well be more consistent for Nessie users to always use the full namespaced names.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521379587



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.ContentsApi;
+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.Branch;
+import com.dremio.nessie.model.Reference;
+import java.io.File;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StructType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public abstract class BaseTestIceberg {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(BaseTestIceberg.class);
+
+  protected static File tempDir;
+  protected NessieCatalog catalog;
+  protected NessieClient client;
+  protected TreeApi tree;
+  protected ContentsApi contents;
+  protected Configuration hadoopConfig;
+  protected final String branch;
+
+  @BeforeClass
+  public static void create() throws Exception {
+    tempDir = java.nio.file.Files.createTempDirectory(
+        "test",
+        PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwxrwx")))
+        .toFile();
+  }
+
+  public BaseTestIceberg(String branch) {
+    this.branch = branch;
+  }
+
+  private void resetData() throws NessieConflictException, NessieNotFoundException {
+    for (Reference r : tree.getAllReferences()) {
+      if (r instanceof Branch) {
+        tree.deleteBranch(r.getName(), r.getHash());
+      } else {
+        tree.deleteTag(r.getName(), r.getHash());
+      }
+    }
+    tree.createReference(Branch.of("main", null));
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    String port = System.getProperty("quarkus.http.test-port", "19120");
+    String path = String.format("http://localhost:%s/api/v1", port);
+    this.client = NessieClient.none(path);
+    tree = client.getTreeApi();
+    contents = client.getContentsApi();
+
+    resetData();
+
+    try {
+      tree.createReference(Branch.of(branch, null));
+    } catch (Exception e) {
+      // ignore, already created. Cant run this in BeforeAll as quarkus hasn't disabled auth
+    }
+
+    hadoopConfig = new Configuration();
+    hadoopConfig.set(NessieClient.CONF_NESSIE_URL, path);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, branch);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_AUTH_TYPE, "NONE");
+    hadoopConfig.set("nessie.warehouse.dir", tempDir.toURI().toString());
+    catalog = initCatalog(branch);
+  }
+
+  NessieCatalog initCatalog(String ref) {
+    NessieCatalog newCatalog = new NessieCatalog();
+    newCatalog.setConf(hadoopConfig);
+    newCatalog.initialize(null, ImmutableMap.of(NessieClient.CONF_NESSIE_REF, ref));
+    return newCatalog;
+  }
+
+  protected Table createTable(TableIdentifier tableIdentifier, int count) {
+    try {
+      return catalog.createTable(tableIdentifier, schema(count));
+    } catch (Throwable t) {
+      LOGGER.error("unable to do create " + tableIdentifier.toString(), t);
+      throw t;
+    }
+  }
+
+  protected void createTable(TableIdentifier tableIdentifier) {
+    Schema schema = new Schema(StructType.of(required(1, "id", LongType.get()))
+                                         .fields());
+    catalog.createTable(tableIdentifier, schema).location();
+  }
+
+  protected static Schema schema(int count) {
+    List<Types.NestedField> fields = new ArrayList<>();
+    for (int i = 0; i < count; i++) {
+      fields.add(required(i, "id" + i, Types.LongType.get()));
+    }
+    return new Schema(Types.StructType.of(fields).fields());
+  }
+
+  void createBranch(String name, String hash) throws NessieNotFoundException, NessieConflictException {
+    if (hash == null) {
+      tree.createReference(Branch.of(name, null));
+    } else {
+      tree.createReference(Branch.of(name, hash));
+    }

Review comment:
       :-) i think you are correct




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521378924



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.ContentsApi;
+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.Branch;
+import com.dremio.nessie.model.Reference;
+import java.io.File;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StructType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public abstract class BaseTestIceberg {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(BaseTestIceberg.class);
+
+  protected static File tempDir;
+  protected NessieCatalog catalog;
+  protected NessieClient client;
+  protected TreeApi tree;
+  protected ContentsApi contents;
+  protected Configuration hadoopConfig;
+  protected final String branch;
+
+  @BeforeClass
+  public static void create() throws Exception {
+    tempDir = java.nio.file.Files.createTempDirectory(
+        "test",
+        PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwxrwx")))
+        .toFile();
+  }
+
+  public BaseTestIceberg(String branch) {
+    this.branch = branch;
+  }
+
+  private void resetData() throws NessieConflictException, NessieNotFoundException {
+    for (Reference r : tree.getAllReferences()) {
+      if (r instanceof Branch) {
+        tree.deleteBranch(r.getName(), r.getHash());
+      } else {
+        tree.deleteTag(r.getName(), r.getHash());
+      }
+    }
+    tree.createReference(Branch.of("main", null));
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    String port = System.getProperty("quarkus.http.test-port", "19120");
+    String path = String.format("http://localhost:%s/api/v1", port);
+    this.client = NessieClient.none(path);
+    tree = client.getTreeApi();
+    contents = client.getContentsApi();
+
+    resetData();
+
+    try {
+      tree.createReference(Branch.of(branch, null));
+    } catch (Exception e) {
+      // ignore, already created. Cant run this in BeforeAll as quarkus hasn't disabled auth
+    }
+
+    hadoopConfig = new Configuration();
+    hadoopConfig.set(NessieClient.CONF_NESSIE_URL, path);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, branch);
+    hadoopConfig.set(NessieClient.CONF_NESSIE_AUTH_TYPE, "NONE");
+    hadoopConfig.set("nessie.warehouse.dir", tempDir.toURI().toString());
+    catalog = initCatalog(branch);
+  }
+
+  NessieCatalog initCatalog(String ref) {
+    NessieCatalog newCatalog = new NessieCatalog();
+    newCatalog.setConf(hadoopConfig);
+    newCatalog.initialize(null, ImmutableMap.of(NessieClient.CONF_NESSIE_REF, ref));
+    return newCatalog;
+  }
+
+  protected Table createTable(TableIdentifier tableIdentifier, int count) {
+    try {
+      return catalog.createTable(tableIdentifier, schema(count));
+    } catch (Throwable t) {
+      LOGGER.error("unable to do create " + tableIdentifier.toString(), t);
+      throw t;
+    }
+  }
+
+  protected void createTable(TableIdentifier tableIdentifier) {
+    Schema schema = new Schema(StructType.of(required(1, "id", LongType.get()))
+                                         .fields());
+    catalog.createTable(tableIdentifier, schema).location();
+  }
+
+  protected static Schema schema(int count) {
+    List<Types.NestedField> fields = new ArrayList<>();
+    for (int i = 0; i < count; i++) {
+      fields.add(required(i, "id" + i, Types.LongType.get()));
+    }
+    return new Schema(Types.StructType.of(fields).fields());
+  }
+
+  void createBranch(String name, String hash) throws NessieNotFoundException, NessieConflictException {
+    if (hash == null) {
+      tree.createReference(Branch.of(name, null));
+    } else {
+      tree.createReference(Branch.of(name, hash));
+    }
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    catalog.close();
+    client.close();
+    catalog = null;
+    client = null;
+    hadoopConfig = null;
+  }
+
+  @AfterClass
+  public static void destroy() throws Exception {
+    tempDir.delete();
+  }
+
+  static String getContent(NessieCatalog catalog, TableIdentifier tableIdentifier) {

Review comment:
       fixed, contents is an internal nessie name for the objects stored in the Nessie object database. Changed contextualize it to content held for iceberg




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527697410



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = NessieUtil.removeCatalogName(toOriginal, name());
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(NessieUtil.toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (NessieConflictException e) {
+      throw new CommitFailedException(e, "failed");

Review comment:
       yup. I believe I was lazy here b/c that exception is always swallowed by the re-try logic above. Not such a good excuse :-) fixed now




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520824084



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    return tableStream(namespace)
+        .map(TableIdentifier::namespace)
+        .filter(n -> !n.isEmpty())
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * namespace metadata is not supported in Nessie and we return an empty map.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return an empty map
+   */
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    return ImmutableMap.of();
+  }
+
+  /**
+   * Namespaces in Nessie are implicit and deleting them results in a no-op.
+   *
+   * @param namespace a namespace. {@link Namespace}
+   * @return always false.
+   */
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    return false;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set namespace properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  @Override
+  public void initialize(String inputName, Map<String, String> options) {
+    this.name = inputName;
+    init(options.getOrDefault(NessieClient.CONF_NESSIE_REF, config.get(NessieClient.CONF_NESSIE_REF)),
+         options.getOrDefault(NessieClient.CONF_NESSIE_URL, config.get(NessieClient.CONF_NESSIE_URL)),
+         options.getOrDefault(NESSIE_WAREHOUSE_DIR, config.get(NESSIE_WAREHOUSE_DIR)));
+  }
+
+  public static class Builder {
+    private final Configuration conf;
+    private String url;
+    private String ref;
+    private String warehouseLocation;
+    private String name;
+
+    public Builder(Configuration conf) {
+      this.conf = conf;
+    }
+
+    public Builder setUrl(String url) {

Review comment:
       Nessie URL? In other places, we configure the connection using `uri`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511959931



##########
File path: build.gradle
##########
@@ -891,6 +924,49 @@ project(':iceberg-pig') {
   }
 }
 
+project(':iceberg-nessie') {
+  apply plugin: 'org.projectnessie'

Review comment:
       It uses `quarkusAppRunnerConfig` dependencies to discover the Nessie Quarkus server and its dependencies then uses that to start a server. Some of the operations to discover all runtime dependencies are non-trivial and require a full gradle dependency graph, hence why its non-trivial to do in a test suite. I believe the primary reason for all this is to facilitate easily building graalvm native images.
   
   See https://github.com/projectnessie/nessie/tree/main/tools/apprunner-gradle-plugin for the actual code

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.client.NessieClient.AuthType;
+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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  public static final String CONF_NESSIE_URL = "nessie.url";
+  public static final String CONF_NESSIE_USERNAME = "nessie.username";
+  public static final String CONF_NESSIE_PASSWORD = "nessie.password";
+  public static final String CONF_NESSIE_AUTH_TYPE = "nessie.auth_type";
+  public static final String NESSIE_AUTH_TYPE_DEFAULT = "BASIC";
+  public static final String CONF_NESSIE_REF = "nessie.ref";
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+    String path = url == null ? config.get(CONF_NESSIE_URL) : url;
+    String username = config.get(CONF_NESSIE_USERNAME);
+    String password = config.get(CONF_NESSIE_PASSWORD);
+    String authTypeStr = config.get(CONF_NESSIE_AUTH_TYPE, NESSIE_AUTH_TYPE_DEFAULT);
+    AuthType authType = AuthType.valueOf(authTypeStr);
+    this.client = new NessieClient(authType, path, username, password);
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");

Review comment:
       I have cleaned this up a bit and tried to follow the pattern you suggested in #1640

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));

Review comment:
       fixed

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+
+    this.client = NessieClient.withConfig(s -> {
+      if (s.equals(NessieClient.CONF_NESSIE_URL)) {
+        return url == null ? config.get(s) : url;
+      }
+      return config.get(s);
+    });
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(NessieClient.CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");
+    if (nessieWarehouseDir != null) {
+      return nessieWarehouseDir;
+    }
+    String hiveWarehouseDir = config.get("hive.metastore.warehouse.dir");
+    if (hiveWarehouseDir != null) {
+      return hiveWarehouseDir;
+    }
+    String defaultFS = config.get("fs.defaultFS");
+    if (defaultFS != null) {
+      return defaultFS + "/" + ICEBERG_HADOOP_WAREHOUSE_BASE;
+    }
+    throw new IllegalStateException("Don't know where to put the nessie iceberg data. " +
+        "Please set one of the following:\n" +
+        "nessie.warehouse.dir\n" +
+        "hive.metastore.warehouse.dir\n" +
+        "fs.defaultFS.");
+  }
+
+  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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier)
+          .collect(Collectors.toList());
+    } catch (NessieNotFoundException ex) {
+      throw new RuntimeException("Unable to list tables due to missing ref.", ex);

Review comment:
       :+1: 

##########
File path: build.gradle
##########
@@ -693,6 +709,8 @@ if (jdkVersion == '8') {
       // Vectorized reads need more memory
       maxHeapSize '2500m'
     }
+    // start and stop quarkus for nessie tests
+    tasks.test.dependsOn("quarkus-start").finalizedBy("quarkus-stop")

Review comment:
       Quarkus (which is the underlying http framework)  behaviour is slightly counterintuitive in that it doesn't offer an option to start Nessie like you can start the hive metastore. Hence we start it once per module and test suites are responsible for cleanup

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.client.NessieClient.AuthType;
+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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {

Review comment:
       I will take another pass at this today, I can see totally valid reasons to support listing namespaces if they have tables in them. The problem as I see it comes from creating or deleting namespaces, and storing namespace metadata.
   
   * create/delete: in Nessie (similar to git) a namespace would be created implicitly with the first table in that namespace tree and deleted with the last table in that namespace tree. Separate crerate/delete options in nessie are either no-ops or require a dummy to be placed in that namespace. Both of which are odd operations. eg if its a no-op then creating namespace `foo.bar` then asking if `foo.bar` exists will return `false`.
   
   * namespace metadata: What is the use case envisioned for those operations? I think for Nessie we would start with the same behaviour as the hdfs catalog but am curious to know the benefit of supporting those apis.

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.client.NessieClient.AuthType;
+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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {

Review comment:
       Having another look we could add valid impls for `namespaceExists` and `listNamespaces` and do no-op or throw for the others. Then the clients can still navigate namespaces. Thoughts?

##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java
##########
@@ -56,6 +56,8 @@ public void cleanNamespaces() {
 
   @Test
   public void testCreateNamespace() {
+    // Nessie namespaces are explicit and do not need to be explicitly managed
+    Assume.assumeFalse(catalogName.endsWith("testnessie"));

Review comment:
       Sure, the hadoop catalog is also skipped for most of these. Makes sense to have separate tests

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+
+    this.client = NessieClient.withConfig(s -> {
+      if (s.equals(NessieClient.CONF_NESSIE_URL)) {
+        return url == null ? config.get(s) : url;
+      }
+      return config.get(s);
+    });
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(NessieClient.CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");
+    if (nessieWarehouseDir != null) {
+      return nessieWarehouseDir;
+    }
+    String hiveWarehouseDir = config.get("hive.metastore.warehouse.dir");
+    if (hiveWarehouseDir != null) {
+      return hiveWarehouseDir;
+    }
+    String defaultFS = config.get("fs.defaultFS");
+    if (defaultFS != null) {
+      return defaultFS + "/" + ICEBERG_HADOOP_WAREHOUSE_BASE;
+    }
+    throw new IllegalStateException("Don't know where to put the nessie iceberg data. " +
+        "Please set one of the following:\n" +
+        "nessie.warehouse.dir\n" +
+        "hive.metastore.warehouse.dir\n" +
+        "fs.defaultFS.");
+  }
+
+  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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier)
+          .collect(Collectors.toList());

Review comment:
       Just checked and the contract is `Return all the identifiers under this namespace.` I took this to mean everything under this and all sub namespaces. If that was not the intention of the method I will fix the predicate.

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {
+
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String ICEBERG_HADOOP_WAREHOUSE_BASE = "iceberg/warehouse";
+  private final NessieClient client;
+  private final String warehouseLocation;
+  private final Configuration config;
+  private final UpdateableReference reference;
+  private final String name;
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config) {
+    this("nessie", config);
+  }
+
+  /**
+   * create a catalog from a hadoop configuration.
+   */
+  public NessieCatalog(Configuration config, String ref) {
+    this("nessie", config, ref);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config) {
+    this(name, config, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref) {
+    this(name, config, ref, null);
+  }
+
+  /**
+   * Create a catalog with a known name from a hadoop configuration.
+   */
+  public NessieCatalog(String name, Configuration config, String ref, String url) {
+    this.config = config;
+    this.name = name;
+
+    this.client = NessieClient.withConfig(s -> {
+      if (s.equals(NessieClient.CONF_NESSIE_URL)) {
+        return url == null ? config.get(s) : url;
+      }
+      return config.get(s);
+    });
+
+    warehouseLocation = getWarehouseLocation();
+
+    final String requestedRef = ref != null ? ref : config.get(NessieClient.CONF_NESSIE_REF);
+    this.reference = get(requestedRef);
+  }
+
+  private String getWarehouseLocation() {
+    String nessieWarehouseDir = config.get("nessie.warehouse.dir");
+    if (nessieWarehouseDir != null) {
+      return nessieWarehouseDir;
+    }
+    String hiveWarehouseDir = config.get("hive.metastore.warehouse.dir");
+    if (hiveWarehouseDir != null) {
+      return hiveWarehouseDir;
+    }
+    String defaultFS = config.get("fs.defaultFS");
+    if (defaultFS != null) {
+      return defaultFS + "/" + ICEBERG_HADOOP_WAREHOUSE_BASE;
+    }
+    throw new IllegalStateException("Don't know where to put the nessie iceberg data. " +
+        "Please set one of the following:\n" +
+        "nessie.warehouse.dir\n" +
+        "hive.metastore.warehouse.dir\n" +
+        "fs.defaultFS.");
+  }
+
+  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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier)
+          .collect(Collectors.toList());

Review comment:
       You are correct, it will return everythiing in and below `namespace`. What is the contract supposed to be? Only tables in this namespace? 

##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java
##########
@@ -50,6 +50,7 @@ public TestCreateTableAsSelect(String catalogName, String implementation, Map<St
   @After
   public void removeTables() {
     sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);

Review comment:
       The way I was running in the test made it get deleted on the backend nessie server but not in the cached spark context I will clean this up as part of the Spark rework

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      this.table = null;
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieNotFoundException | NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new CommitFailedException(ex, "failed");
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private static String applicationId() {
+    try {
+      if (sparkConfMethod == null) {
+        Class sparkEnvClazz = Class.forName("org.apache.spark.SparkEnv");
+        sparkEnvMethod = sparkEnvClazz.getMethod("get");
+        Class sparkConfClazz = Class.forName("org.apache.spark.SparkConf");
+        sparkConfMethod = sparkEnvClazz.getMethod("conf");
+        appIdMethod = sparkConfClazz.getMethod("getAppId");

Review comment:
       :+1: 
   

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      this.table = null;
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),

Review comment:
       good eye, the first char of the `applicationId` is a newline. I've put no space between `commit` and `%s` to not have extra trailing whitespace in message.
   
   Also note that the handling of commit messages in nessie is still fairly primitive. This should get replaced by a structured object in the near future.

##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestParsedTableIdentifier.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.client.NessieClient;
+import java.util.HashMap;
+import java.util.Map;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestParsedTableIdentifier {
+
+
+  @Test
+  public void noMarkings() {
+    String path = "foo";
+    ParsedTableIdentifier pti = ParsedTableIdentifier.getParsedTableIdentifier(path, new HashMap<>());
+    Assert.assertEquals(path, pti.getTableIdentifier().name());
+    Assert.assertNull(pti.getReference());
+    Assert.assertNull(pti.getTimestamp());
+  }
+
+  @Test
+  public void branchOnly() {
+    String path = "foo@bar";
+    ParsedTableIdentifier pti = ParsedTableIdentifier.getParsedTableIdentifier(path, new HashMap<>());
+    Assert.assertEquals("foo", pti.getTableIdentifier().name());
+    Assert.assertEquals("bar", pti.getReference());
+    Assert.assertNull(pti.getTimestamp());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void timestampOnly() {
+    String path = "foo#baz";
+    ParsedTableIdentifier.getParsedTableIdentifier(path, new HashMap<>());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void branchAndTimestamp() {
+    String path = "foo@bar#baz";
+    ParsedTableIdentifier.getParsedTableIdentifier(path, new HashMap<>());
+  }
+
+  @Test(expected = IllegalArgumentException.class)

Review comment:
       fixed

##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      this.table = null;
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieNotFoundException | NessieConflictException ex) {

Review comment:
       good eye, cleaned up exception message and handled throwing better




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521398167



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/ParsedTableIdentifier.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.client.NessieClient;
+import java.time.Instant;
+import java.util.Map;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+public class ParsedTableIdentifier {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * container class to hold all options in a Nessie table name.
+   */
+  public ParsedTableIdentifier(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(TableIdentifier path) {
+    return getParsedTableIdentifier(path, ImmutableMap.of());
+  }
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static ParsedTableIdentifier getParsedTableIdentifier(String path, Map<String, String> properties) {

Review comment:
       correct, even further the ref isn't really needed. Thanks for pointing that out, much cleaner now.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526544015



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalogBranch.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalogBranch extends BaseTestIceberg {

Review comment:
       What is the main difference between this suite and the branch suite above? Seems very similar to me, so I think I've missed the point. Smaller test cases that are well named would help.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r528929656



##########
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));

Review comment:
       It is just ref now, the `removePrefix` method strips the `nessie.`from the constant in the nessie class. Didn't want to duplicate the constants already in `NessieClient`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527679913



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);
+    if (warehouseLocation == null) {
+      throw new IllegalStateException("Parameter nessie.warehouse.dir not set, nessie can't store data.");
+    }
+    final String requestedRef = options.get(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.getReference() != null) {
+      newReference = loadReference(pti.getReference());
+    }
+    return new NessieTableOperations(NessieUtil.toKey(pti.getTableIdentifier()), 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.
+    int count = 0;
+    while (count < 5) {
+      count++;
+      try {
+        dropTableInner(identifier);
+        break;
+      } catch (NessieConflictException e) {
+        // pass for retry
+      } catch (NessieNotFoundException e) {
+        logger.error("Cannot drop table: ref is no longer valid.", e);
+        return false;
+      }
+    }
+    if (count >= 5) {
+      logger.error("Cannot drop table: failed after retry (update hash and retry)");
+      return false;
+    }

Review comment:
       :+1: I have updated to use Tasks. Fun class :-) 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521617869



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private String applicationId() {
+    String appId = null;
+    TableMetadata current = current();
+    if (current != null) {
+      Snapshot snapshot = current.currentSnapshot();
+      if (snapshot != null) {
+        Map<String, String> summary = snapshot.summary();
+        appId = summary.get("spark.app.id");

Review comment:
       I think we should update our SparkCatalog to pass information into catalogs.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526539150



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private String applicationId() {
+    String appId = null;
+    TableMetadata current = current();
+    if (current != null) {
+      Snapshot snapshot = current.currentSnapshot();
+      if (snapshot != null) {
+        Map<String, String> summary = snapshot.summary();
+        appId = summary.get("spark.app.id");

Review comment:
       Yeah, we should copy and add it. Let's go forward with this for now and we can fix it in a follow-up.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-732350207


   @rymurr, thanks for all of the test changes, it is now much easier to understand! I don't see any blockers, although it looks like you may have intended to change the `nessie.ref` config to just `ref`. I'm going to go ahead and merge this since it is ready and we can clean that up later if you want to change it.
   
   Thanks for all your hard work getting this ready! I actually quite like the way the Nessie reference works and simplifies assumptions in the catalog and table operations.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526070852



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);

Review comment:
       done 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526529343



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;

Review comment:
       Minor: It seems strange to me to default this in the catalog rather than in the tests. I would probably use a precondition to validate it isn't null instead.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527829234



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+
+    Table bar = createTable(foobar, 1); // table 1
+    catalog.refresh();

Review comment:
       Hm. The commit is going to trigger a refresh on the table the next time its metadata is accessed. That is usually right away because operations will read the latest metadata to clean up unused files. For Nessie, `doRefresh` is going to update the shared ref. That's why I thought this was redundant. I guess there could be cases where the metadata isn't accessed yet and you do need the refresh.
   
   Not a big deal, so let's not worry about it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526543001



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestBranchHash.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBranchHash extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-branch-hash";
+
+  public TestBranchHash() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+
+    Table bar = createTable(foobar, 1); // table 1
+    catalog.refresh();
+    createBranch("test", catalog.currentHash());
+
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "test");
+
+    NessieCatalog newCatalog = initCatalog("test");
+    String initialMetadataLocation = metadataLocation(newCatalog, foobar);
+    Assert.assertEquals(initialMetadataLocation, metadataLocation(catalog, foobar));
+
+    bar.updateSchema().addColumn("id1", Types.LongType.get()).commit();
+
+    // metadata location changed no longer matches
+    Assert.assertNotEquals(metadataLocation(catalog, foobar), metadataLocation(newCatalog, foobar));
+
+    // points to the previous metadata location
+    Assert.assertEquals(initialMetadataLocation, metadataLocation(newCatalog, foobar));
+
+
+    String mainHash = tree.getReferenceByName(BRANCH).getHash();
+    // catalog created with ref and no hash points to same catalog as above
+    NessieCatalog refCatalog = initCatalog("test");
+    Assert.assertEquals(metadataLocation(newCatalog, foobar), metadataLocation(refCatalog, foobar));
+    // catalog created with ref and hash points to
+    NessieCatalog refHashCatalog = initCatalog(mainHash);
+    Assert.assertEquals(metadataLocation(catalog, foobar), metadataLocation(refHashCatalog, foobar));
+
+    // asking for table@branch gives expected regardless of catalog
+    Assert.assertEquals(metadataLocation(newCatalog, foobar),
+        metadataLocation(catalog, TableIdentifier.of("foo", "bar@test")));
+    // asking for table@branch#hash gives expected regardless of catalog
+    Assert.assertEquals(metadataLocation(catalog, foobar),
+        metadataLocation(catalog, TableIdentifier.of("foo", "bar@" + mainHash)));

Review comment:
       This is another test I think could be broken into distinct cases with a `@Before` to set up the default branch and table.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511179235



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      this.table = null;
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieNotFoundException | NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new CommitFailedException(ex, "failed");
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private static String applicationId() {
+    try {
+      if (sparkConfMethod == null) {
+        Class sparkEnvClazz = Class.forName("org.apache.spark.SparkEnv");
+        sparkEnvMethod = sparkEnvClazz.getMethod("get");
+        Class sparkConfClazz = Class.forName("org.apache.spark.SparkConf");
+        sparkConfMethod = sparkEnvClazz.getMethod("conf");
+        appIdMethod = sparkConfClazz.getMethod("getAppId");

Review comment:
       You can use the `DynFields` helpers to do this a bit more easily.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520909112



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    boolean threw = true;
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+      threw = false;
+    } catch (NessieConflictException ex) {
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } finally {
+      if (threw) {
+        io().deleteFile(newMetadataLocation);
+      }
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }

Review comment:
       We've moved the other implementations to create a FileIO at the catalog level and pass it into TableOperations. You may want to do the same. Also, you'll probably want to update to use the same logic so that the implementation can be overridden dynamically.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520828053



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();

Review comment:
       This doesn't seem correct to me. I try to maintain single-table state by catalog, so that all uses of a table stay in sync. I think it would make sense to do the same with refs. If you update a branch by refreshing or committing any table, it should also refresh everything that is related to stay in sync. Otherwise, you're left with the problem of not knowing whether two tables with the same ref are in sync.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526539705



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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 java.time.Instant;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+public class TableReference {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * Container class to specify a TableIdentifier on a specific Reference or at an Instant in time.
+   */
+  public TableReference(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {

Review comment:
       Nit: We usually omit `get` from getter method names because it doesn't add value and looks weird in other languages (like Scala and Kotlin).




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-729831234


   I'll take another look. Thanks for the updates!


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511180243



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestParsedTableIdentifier.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.client.NessieClient;
+import java.util.HashMap;
+import java.util.Map;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestParsedTableIdentifier {
+
+
+  @Test
+  public void noMarkings() {
+    String path = "foo";
+    ParsedTableIdentifier pti = ParsedTableIdentifier.getParsedTableIdentifier(path, new HashMap<>());
+    Assert.assertEquals(path, pti.getTableIdentifier().name());
+    Assert.assertNull(pti.getReference());
+    Assert.assertNull(pti.getTimestamp());
+  }
+
+  @Test
+  public void branchOnly() {
+    String path = "foo@bar";
+    ParsedTableIdentifier pti = ParsedTableIdentifier.getParsedTableIdentifier(path, new HashMap<>());
+    Assert.assertEquals("foo", pti.getTableIdentifier().name());
+    Assert.assertEquals("bar", pti.getReference());
+    Assert.assertNull(pti.getTimestamp());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void timestampOnly() {
+    String path = "foo#baz";
+    ParsedTableIdentifier.getParsedTableIdentifier(path, new HashMap<>());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void branchAndTimestamp() {
+    String path = "foo@bar#baz";
+    ParsedTableIdentifier.getParsedTableIdentifier(path, new HashMap<>());
+  }
+
+  @Test(expected = IllegalArgumentException.class)

Review comment:
       We prefer using `AssertHelpers.assertThrows` so that state after the exception was thrown can be validated. For example, testing `catalog.createTable(invalid)` would not only check `ValidationException` but also verify that the table was not created.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-729690343


   Hey @rdblue, responded to some of your comments. Just a few more things to sort out now :-)


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-730079214


   @rymurr, I did another thorough review with more time looking through the tests. Looking close, but I found a few things.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520842724



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");

Review comment:
       correct, now refreshes and tries again




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520900487



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/ParsedTableIdentifier.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.client.NessieClient;
+import java.time.Instant;
+import java.util.Map;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+public class ParsedTableIdentifier {

Review comment:
       I'm wondering if there is a more specific name for this class. Maybe something like `TableReference` because it has both an identifier and a ref? Or maybe `NessieIdentifier`? `ParsedTableIdentifier` doesn't really tell me what is different about this as opposed to `TableIdentifier`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520823402



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.
+    refresh();
+    return true;
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier toOriginal) {
+    reference.checkMutable();
+
+    TableIdentifier to = removeCatalogName(toOriginal);
+
+    IcebergTable existingFromTable = table(from);
+    if (existingFromTable == null) {
+      throw new NoSuchTableException("table %s doesn't exists", from.name());
+    }
+    IcebergTable existingToTable = table(to);
+    if (existingToTable != null) {
+      throw new AlreadyExistsException("table %s already exists", to.name());
+    }
+
+    Operations contents = ImmutableOperations.builder()
+        .addOperations(ImmutablePut.builder().key(toKey(to)).contents(existingFromTable).build(),
+            ImmutableDelete.builder().key(toKey(from)).build())
+        .build();
+
+    try {
+      client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(),
+          "iceberg rename table", contents);
+      // TODO: fix this so we don't depend on it in tests.
+      refresh();
+    } catch (Exception e) {
+      throw new CommitFailedException(e, "failed");
+    }
+  }
+
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+
+    String[] levels = to.namespace().levels();
+    // check if the identifier includes the catalog name and remove it
+    if (levels.length >= 2 && name().equalsIgnoreCase(to.namespace().level(0))) {
+      Namespace trimmedNamespace = Namespace.of(Arrays.copyOfRange(levels, 1, levels.length));
+      return TableIdentifier.of(trimmedNamespace, to.name());
+    }
+
+    // return the original unmodified
+    return to;
+  }
+
+  public TreeApi getTreeApi() {
+    return client.getTreeApi();
+  }
+
+  public void refresh() {
+    reference.refresh();
+  }
+
+  public String getHash() {
+    return reference.getHash();
+  }
+
+  public static Builder builder(Configuration conf) {
+    return new Builder(conf);
+  }
+
+  /**
+   * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored.
+   *
+   * @param namespace a multi-part namespace
+   * @param metadata a string Map of properties for the given namespace
+   */
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {

Review comment:
       Should we create a trait just for listing namespaces that are implicit?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527713506



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private FileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client,
+      FileIO fileIO) {
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+    this.fileIO = fileIO;
+  }
+
+  @Override
+  protected void doRefresh() {
+    try {
+      reference.refresh();
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to refresh as ref is no longer valid.", e);

Review comment:
       I would normally agree but `NessieNotFoundException` is a checked exception and I don't want to add that throws clause to the interface. Is there a more specialised unchecked exception that you think would suit?
   
   If it helps the likelihood of this throwing should be very low in real life. Typically a user/service would own their branches and deleting a branch when someone else is actively working on it would be an organisational issue rather than something that should be handled in code. An analogy in git would be someone deleting your branch which is an active PR (without closing the PR).




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527822741



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;

Review comment:
       If that's what the other catalogs do, then that seems reasonable. I think we eventually want to guarantee that there is always a name, but there are probably cases where we didn't have one. We can look into that separately.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527849628



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalogBranch.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalogBranch extends BaseTestIceberg {
+
+  public TestCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+    TableIdentifier foobaz = TableIdentifier.of("foo", "baz");
+    Table bar = createTable(foobar, 1); // table 1
+    createTable(foobaz, 1); // table 2
+    catalog.refresh();
+    createBranch("test", catalog.currentHash());
+
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "test");
+
+    NessieCatalog newCatalog = initCatalog("test");
+    String initialMetadataLocation = metadataLocation(newCatalog, foobar);
+    Assert.assertEquals(initialMetadataLocation, metadataLocation(catalog, foobar));
+    Assert.assertEquals(metadataLocation(newCatalog, foobaz), metadataLocation(catalog, foobaz));

Review comment:
       ive re-worked this test class. Let me know if thats easier to read




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521398634



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java
##########
@@ -0,0 +1,349 @@
+/*
+ * 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.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.ContentsKey;
+import com.dremio.nessie.model.IcebergTable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadataParser;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableMetadataParser.getFileExtension;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestNessieTable extends BaseTestIceberg {
+
+  private static final String BRANCH = "iceberg-table-test";
+
+  private static final String DB_NAME = "db";
+  private static final String TABLE_NAME = "tbl";
+  private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME);
+  private static final ContentsKey KEY = ContentsKey.of(DB_NAME, TABLE_NAME);
+  private static final Schema schema = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get())).fields());
+  private static final Schema altered = new Schema(Types.StructType.of(
+      required(1, "id", Types.LongType.get()),
+      optional(2, "data", Types.LongType.get())).fields());
+
+  private Path tableLocation;
+
+  public TestNessieTable() {
+    super(BRANCH);
+  }
+
+  @Before
+  public void beforeEach() throws NessieConflictException, NessieNotFoundException {
+    super.beforeEach();
+    this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema).location());
+  }
+
+  @After
+  public void afterEach() throws Exception {
+    // drop the table data
+    if (tableLocation != null) {
+      tableLocation.getFileSystem(hadoopConfig).delete(tableLocation, true);
+      catalog.refresh();
+      catalog.dropTable(TABLE_IDENTIFIER, false);
+    }
+
+    super.afterEach();
+  }
+
+  private com.dremio.nessie.model.IcebergTable getTable(ContentsKey key) throws NessieNotFoundException {
+    return client.getContentsApi()
+        .getContents(key, BRANCH)
+        .unwrap(IcebergTable.class).get();
+  }
+
+  @Test
+  public void testCreate() throws NessieNotFoundException {
+    // Table should be created in iceberg
+    // Table should be renamed in iceberg
+    String tableName = TABLE_IDENTIFIER.name();
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+    // add a column
+    icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit();
+    IcebergTable table = getTable(KEY);
+    // check parameters are in expected state
+    Assert.assertEquals(getTableLocation(tableName),
+                            (tempDir.toURI().toString() + DB_NAME + "/" +
+                             tableName).replace("//",
+                                                "/"));
+
+    // Only 1 snapshotFile Should exist and no manifests should exist
+    Assert.assertEquals(2, metadataVersionFiles(tableName).size());
+    Assert.assertEquals(0, manifestFiles(tableName).size());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testRename() {
+    String renamedTableName = "rename_table_name";
+    TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(),
+                                                               renamedTableName);
+
+    Table original = catalog.loadTable(TABLE_IDENTIFIER);
+
+    catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier);
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.tableExists(renameTableIdentifier));
+
+    Table renamed = catalog.loadTable(renameTableIdentifier);
+
+    Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct());
+    Assert.assertEquals(original.spec(), renamed.spec());
+    Assert.assertEquals(original.location(), renamed.location());
+    Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot());
+
+    Assert.assertTrue(catalog.dropTable(renameTableIdentifier));
+  }
+
+  @Test
+  public void testDrop() {
+    Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER));
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+  }
+
+  @Test
+  public void testDropWithoutPurgeLeavesTableData() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String fileLocation = table.location().replace("file:", "") + "/data/file.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(fileLocation))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    DataFile file = DataFiles.builder(table.spec())
+                             .withRecordCount(3)
+                             .withPath(fileLocation)
+                             .withFileSizeInBytes(Files.localInput(fileLocation).getLength())
+                             .build();
+
+    table.newAppend().appendFile(file).commit();
+
+    String manifestListLocation =
+        table.currentSnapshot().manifestListLocation().replace("file:", "");
+
+    Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false));
+    Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER));
+
+    Assert.assertTrue(new File(fileLocation).exists());
+    Assert.assertTrue(new File(manifestListLocation).exists());
+  }
+
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testDropTable() throws IOException {
+    Table table = catalog.loadTable(TABLE_IDENTIFIER);
+
+    GenericRecordBuilder recordBuilder =
+        new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test"));
+    List<GenericData.Record> records = new ArrayList<>();
+    records.add(recordBuilder.set("id", 1L).build());
+    records.add(recordBuilder.set("id", 2L).build());
+    records.add(recordBuilder.set("id", 3L).build());
+
+    String location1 = table.location().replace("file:", "") + "/data/file1.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location1))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {
+      for (GenericData.Record rec : records) {
+        writer.add(rec);
+      }
+    }
+
+    String location2 = table.location().replace("file:", "") + "/data/file2.avro";
+    try (FileAppender<GenericData.Record> writer = Avro.write(Files.localOutput(location2))
+                                                       .schema(schema)
+                                                       .named("test")
+                                                       .build()) {

Review comment:
       :+1: 
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521615910



##########
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);

Review comment:
       Why remove it in a follow-up? I'd be concerned about not remembering and then needing to break behavior later.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520837127



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);

Review comment:
       I am not 100% certain that `RuntimeException` is the best avenue here, its definitely an unexpected error and in a sense unrecoverable. There are no exceptions referenced in the javadoc so perhaps a log line and returning `false` is more appropriate?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527722629



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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 java.time.Instant;
+import org.apache.iceberg.catalog.TableIdentifier;
+
+public class TableReference {
+
+  private final TableIdentifier tableIdentifier;
+  private final Instant timestamp;
+  private final String reference;
+
+  /**
+   * Container class to specify a TableIdentifier on a specific Reference or at an Instant in time.
+   */
+  public TableReference(TableIdentifier tableIdentifier, Instant timestamp, String reference) {
+    this.tableIdentifier = tableIdentifier;
+    this.timestamp = timestamp;
+    this.reference = reference;
+  }
+
+  public TableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Instant getTimestamp() {
+    return timestamp;
+  }
+
+  public String getReference() {
+    return reference;
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static TableReference parse(TableIdentifier path) {
+    TableReference pti = parse(path.name());
+    return new TableReference(TableIdentifier.of(path.namespace(), pti.getTableIdentifier().name()),
+        pti.getTimestamp(),
+        pti.getReference());
+  }
+
+  /**
+   * Convert dataset read/write options to a table and ref/hash.
+   */
+  public static TableReference parse(String path) {
+    // I am assuming tables can't have @ or # symbols
+    if (path.split("@").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one branch in %s", path));
+    }
+    if (path.split("#").length > 2) {
+      throw new IllegalArgumentException(String.format("Can only reference one timestamp in %s", path));
+    }
+
+    if (path.contains("@") && path.contains("#")) {
+      throw new IllegalArgumentException("Invalid table name:" +
+          " # is not allowed (reference by timestamp is not supported)");
+    }
+
+    if (path.contains("@")) {
+      String[] tableRef = path.split("@");
+      TableIdentifier identifier = TableIdentifier.parse(tableRef[0]);
+      return new TableReference(identifier, null, tableRef[1]);
+    }
+
+    if (path.contains("#")) {
+      throw new IllegalArgumentException("Invalid table name:" +
+          " # is not allowed (reference by timestamp is not supported)");
+    }

Review comment:
       If it doesn't matter much to you I would like to leave it in. I will be adding timestamp support here very soon.
   
   Happy to remove it till then if you prefer too.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527850145



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestDefaultCatalogBranch.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.client.NessieClient;
+import com.dremio.nessie.error.NessieConflictException;
+import com.dremio.nessie.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * test tag operations with a default tag set by server.
+ */
+public class TestDefaultCatalogBranch extends BaseTestIceberg {
+
+  public TestDefaultCatalogBranch() {
+    super("main");
+  }
+
+  @SuppressWarnings("VariableDeclarationUsageDistance")
+  @Test
+  public void testBasicBranch() throws NessieNotFoundException, NessieConflictException {
+    TableIdentifier foobar = TableIdentifier.of("foo", "bar");
+    TableIdentifier foobaz = TableIdentifier.of("foo", "baz");
+    createTable(foobar, 1); // table 1
+    createTable(foobaz, 1); // table 2
+
+    catalog.refresh();
+    tree.createReference(Branch.of("FORWARD", catalog.currentHash()));
+    hadoopConfig.set(NessieClient.CONF_NESSIE_REF, "FORWARD");
+    NessieCatalog forwardCatalog = initCatalog("FORWARD");
+    forwardCatalog.loadTable(foobaz).updateSchema().addColumn("id1", Types.LongType.get()).commit();
+    forwardCatalog.loadTable(foobar).updateSchema().addColumn("id1", Types.LongType.get()).commit();
+    Assert.assertNotEquals(metadataLocation(forwardCatalog, foobar),
+                               metadataLocation(catalog, foobar));
+    Assert.assertNotEquals(metadataLocation(forwardCatalog, foobaz),
+                               metadataLocation(catalog, foobaz));
+
+    System.out.println(metadataLocation(forwardCatalog, foobar));
+    System.out.println(metadataLocation(catalog, foobar));
+
+    forwardCatalog.refresh();
+    tree.assignBranch("main",
+        tree.getReferenceByName("main").getHash(),
+        Branch.of("main", forwardCatalog.currentHash()));
+
+    catalog.refresh();
+
+    System.out.println(metadataLocation(forwardCatalog, foobar));
+    System.out.println(metadataLocation(catalog, foobar));
+
+    Assert.assertEquals(metadataLocation(forwardCatalog, foobar),
+                            metadataLocation(catalog, foobar));
+    Assert.assertEquals(metadataLocation(forwardCatalog, foobaz),
+                            metadataLocation(catalog, foobaz));
+
+    catalog.dropTable(foobar);
+    catalog.dropTable(foobaz);
+    tree.deleteBranch("FORWARD", tree.getReferenceByName("FORWARD").getHash());

Review comment:
       agreed, deleted




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526541193



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalog.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 java.util.List;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalog extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-catalog-branch";
+
+  public TestCatalog() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void test() {
+    createTable(TableIdentifier.of("foo", "bar"));
+    List<TableIdentifier> tables = catalog.listTables(Namespace.of("foo"));
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("bar", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.renameTable(TableIdentifier.of("foo", "bar"), TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(null);
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("baz", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.dropTable(TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(Namespace.empty());
+    Assert.assertTrue(tables.isEmpty());

Review comment:
       Looks good.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520819559



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);

Review comment:
       Is this assuming that the `NessieNotFoundException` is referring to the ref because the table was loaded just above? Or is that always used for a ref?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r519081508



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.common.DynFields;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static DynFields.StaticField<Object> sparkEnvMethod;
+  private static DynFields.UnboundField<Object> sparkConfMethod;
+  private static DynFields.UnboundField<Object> appIdMethod;
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+    } catch (NessieConflictException ex) {
+      io().deleteFile(newMetadataLocation);
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);
+    } catch (NessieNotFoundException ex) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException(String.format("Commit failed: Reference %s does not exist", reference.getName()), ex);
+    } catch (Throwable e) {
+      io().deleteFile(newMetadataLocation);
+      throw new RuntimeException("Unexpected commit exception", e);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    if (fileIO == null) {
+      fileIO = new HadoopFileIO(conf);
+    }
+
+    return fileIO;
+  }
+
+  /**
+   * try and get a Spark application id if one exists.
+   *
+   * <p>
+   *   We haven't figured out a general way to pass commit messages through to the Nessie committer yet.
+   *   This is hacky but gets the job done until we can have a more complete commit/audit log.
+   * </p>
+   */
+  private static String applicationId() {

Review comment:
       For Spark writes, we pass the application ID in through `snapshot.summary()`: https://github.com/apache/iceberg/blob/9af545ed56343b2fa09966166f8da3e7a24100d7/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java#L153




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-732371385


   Thanks for the merge @rdblue!! Super pumped to have this merged. I have the last round of changes ready and will post them with the PR to support timestamps in the table name asap.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r526530130



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.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.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.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("/");
+  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;
+  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;
+    this.client = NessieClient.withConfig(options::get);
+
+    this.warehouseLocation = options.get(NESSIE_WAREHOUSE_DIR);

Review comment:
       Other catalogs use "warehouse" instead of a catalog-specific property. It would be slightly better for consistency to do the same, although it is fine this way. If the `NessieClient` is expecting config like this, it may well be more consistent for Nessie users to always use the full namespaced names.
   
   Just be aware that this will require properties like `spark.sql.catalog.some_name.nessie.warehouse.dir=...`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520844399



##########
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<>());
+    UpdateableReference newReference = this.reference;
+    if (pti.getReference() != null) {
+      newReference = get(pti.getReference());
+    }
+    return new NessieTableOperations(config,
+                                     toKey(pti.getTableIdentifier()),
+                                     newReference,
+                                     client);
+  }
+
+  @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());
+  }
+
+  private Stream<TableIdentifier> tableStream(Namespace namespace) {
+    try {
+      return client.getTreeApi()
+          .getEntries(reference.getHash())
+          .getEntries()
+          .stream()
+          .filter(namespacePredicate(namespace))
+          .map(NessieCatalog::toIdentifier);
+    } catch (NessieNotFoundException ex) {
+      throw new NoSuchNamespaceException(ex, "Unable to list tables due to missing ref. %s", reference.getName());
+    }
+  }
+
+  private static Predicate<EntriesResponse.Entry> namespacePredicate(Namespace ns) {
+    // TODO: filter to just iceberg tables.
+    if (ns == null) {
+      return e -> true;
+    }
+
+    final List<String> namespace = Arrays.asList(ns.levels());
+    Predicate<EntriesResponse.Entry> predicate = e -> {
+      List<String> names = e.getName().getElements();
+
+      if (names.size() <= namespace.size()) {
+        return false;
+      }
+
+      return namespace.equals(names.subList(0, namespace.size()));
+    };
+    return predicate;
+  }
+
+  private static TableIdentifier toIdentifier(EntriesResponse.Entry entry) {
+    List<String> elements = entry.getName().getElements();
+    return TableIdentifier.of(elements.toArray(new String[elements.size()]));
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    reference.checkMutable();
+
+    IcebergTable existingTable = table(identifier);
+    if (existingTable == null) {
+      return false;
+    }
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata;
+    if (purge && ops.current() != null) {
+      lastMetadata = ops.current();
+    } else {
+      lastMetadata = null;
+    }
+
+    try {
+      client.getContentsApi().deleteContents(toKey(identifier), reference.getAsBranch().getName(), reference.getHash(),
+          "no message");
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to drop table as ref is no longer valid.", e);
+    } catch (NessieConflictException e) {
+      throw new RuntimeException("Failed to drop table as table state needs to be refreshed.");
+    }
+
+    // TODO: purge should be blocked since nessie will clean through other means.
+    if (purge && lastMetadata != null) {
+      BaseMetastoreCatalog.dropTableData(ops.io(), lastMetadata);
+    }
+    // TODO: fix this so we don't depend on it in tests.

Review comment:
       We don't strictly need to refresh immediately after an operation. This generates an extra call to the backend which typically isn't required. We have to do it because tests do require it. Tests tend to switch branches, perform multiple actions and make several conflicting changes in short order in the same jvm so need explicit refresh. Since the api call isn't expensive we have left the refresh in until a better strategy (or better tests) are devised




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511173716



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.client.NessieClient.AuthType;
+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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {

Review comment:
       Should be fine, but I think the trade-off is that you won't be able to list namespaces in a namespace. It will be harder to find the namespaces themselves.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: DRAFT: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r503397814



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.client.NessieClient.AuthType;
+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.ImmutableMultiContents;
+import com.dremio.nessie.model.ImmutablePut;
+import com.dremio.nessie.model.MultiContents;
+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.function.Predicate;
+import java.util.stream.Collectors;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+
+/**
+ * Nessie implementation of Iceberg Catalog.
+ */
+public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable {

Review comment:
       We do not extend `SupportsNamespaces` as a Nessie object store supports the concept of namespaces implicitly. A Nessie namespace can be arbitrarily deep but is not explicitly created or stored. Similar to empty folders in git.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511178717



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      this.table = null;

Review comment:
       I think this should throw `NoSuchTableException` if the existing metadata is not null because the table was deleted under the reference. You'll probably want to follow the same behavior as the Hive catalog.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r511177676



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.lang.reflect.Method;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private static Method sparkConfMethod;
+  private static Method appIdMethod;
+  private static Method sparkEnvMethod;
+
+  private final Configuration conf;
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private HadoopFileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      Configuration conf,
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client) {
+    this.conf = conf;
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+  }
+
+  @Override
+  protected void doRefresh() {
+    // break reference with parent (to avoid cross-over refresh)
+    // TODO, confirm this is correct behavior.
+    // reference = reference.copy();
+
+    reference.refresh();
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() -> new IllegalStateException("Nessie points to a non-Iceberg object for that path."));

Review comment:
       Style: Most Iceberg error messages use the form `Cannot <some action>: <reason> (<workaround>)`. Consistency here tends to make at least Iceberg errors more readable and easy to consume.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #1587: Nessie support for core and Spark 2/3

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#issuecomment-716732200


   Hey @rdblue I have addressed the bulk of your comments above. Left to do:
   
   1) make decision on namespace support for nessie catalog
   2) the meaning of `listTables` in Catalog
   3) revised `NessieCatalog` constructor along the lines of your comment in #1640 
   
   We should be publishing 0.2.0 of nessie in the next day or two. Once that is pushed I will update this PR with the new versions and we should have a green build.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521377267



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/UpdateableReference.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.error.NessieNotFoundException;
+import com.dremio.nessie.model.Branch;
+import com.dremio.nessie.model.Hash;
+import com.dremio.nessie.model.Reference;
+import org.apache.iceberg.exceptions.NotFoundException;
+
+class UpdateableReference {
+
+  private Reference reference;
+  private final TreeApi client;
+
+  UpdateableReference(Reference reference, TreeApi client) {
+    this.reference = reference;
+    this.client = client;
+  }
+
+  public boolean refresh() {
+    if (reference instanceof Hash) {
+      return false;
+    }
+    Reference oldReference = reference;
+    try {
+      reference = client.getReferenceByName(reference.getName());
+    } catch (NessieNotFoundException e) {
+      throw new NotFoundException(e, "Failure refreshing data, table no longer exists.");

Review comment:
       fixed the comment and simplified the exception handling




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521367979



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/ParsedTableIdentifier.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.client.NessieClient;
+import java.time.Instant;
+import java.util.Map;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+
+public class ParsedTableIdentifier {

Review comment:
       I chose `TableReference` in the end. I don't love it but its way better than `ParsedTableIdentifier`. I don't know how to concisely state that its a `TableIdentifier` tied to a specific `Reference`. I have updated the javadoc which I hope clarifies it




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r520813255



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

Review comment:
       #1640 is in. It uses a no-arg constructor and adds an `initialize(String name, Map<String, String> config)` method to initialize and configure the catalog. I think you should be able to update this now.
   
   I'm hoping that this removes the need to make Spark and Flink depend on the new Nessie and Glue modules. We should make sure we have a test suite we can include here that uses Flink and Spark.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r527716765



##########
File path: nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.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.IcebergTable;
+import com.dremio.nessie.model.ImmutableIcebergTable;
+import java.util.Map;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+
+/**
+ * Nessie implementation of Iceberg TableOperations.
+ */
+public class NessieTableOperations extends BaseMetastoreTableOperations {
+
+  private final NessieClient client;
+  private final ContentsKey key;
+  private UpdateableReference reference;
+  private IcebergTable table;
+  private FileIO fileIO;
+
+  /**
+   * Create a nessie table operations given a table identifier.
+   */
+  public NessieTableOperations(
+      ContentsKey key,
+      UpdateableReference reference,
+      NessieClient client,
+      FileIO fileIO) {
+    this.key = key;
+    this.reference = reference;
+    this.client = client;
+    this.fileIO = fileIO;
+  }
+
+  @Override
+  protected void doRefresh() {
+    try {
+      reference.refresh();
+    } catch (NessieNotFoundException e) {
+      throw new RuntimeException("Failed to refresh as ref is no longer valid.", e);
+    }
+    String metadataLocation = null;
+    try {
+      Contents contents = client.getContentsApi().getContents(key, reference.getHash());
+      this.table = contents.unwrap(IcebergTable.class)
+          .orElseThrow(() ->
+              new IllegalStateException("Cannot refresh iceberg table: " +
+                  String.format("Nessie points to a non-Iceberg object for path: %s.", key)));
+      metadataLocation = table.getMetadataLocation();
+    } catch (NessieNotFoundException ex) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchTableException(ex, "No such table %s", key);
+      }
+    }
+    refreshFromMetadataLocation(metadataLocation, 2);
+  }
+
+  @Override
+  protected void doCommit(TableMetadata base, TableMetadata metadata) {
+    reference.checkMutable();
+
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+
+    boolean threw = true;
+    try {
+      IcebergTable newTable = ImmutableIcebergTable.builder().metadataLocation(newMetadataLocation).build();
+      client.getContentsApi().setContents(key,
+                                          reference.getAsBranch().getName(),
+                                          reference.getHash(),
+                                          String.format("iceberg commit%s", applicationId()),
+                                          newTable);
+      threw = false;
+    } catch (NessieConflictException ex) {
+      String fixMsg = reference.isBranch() ?
+          String.format("Update the reference %s and try again", reference.getName()) :
+          String.format("Can't commit to the tag %s", reference.getName());
+      throw new CommitFailedException(ex, "Commit failed: Reference hash is out of date. %s", fixMsg);

Review comment:
       Yep! exactly! On line 87 we do `reference.checkMutable()` which will throw `IllegalArgumentException` if its a tag. So by the time we commit we know its a branch. I have cleaned up the erroneous error msg.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #1587: Nessie support for core

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #1587:
URL: https://github.com/apache/iceberg/pull/1587#discussion_r521385726



##########
File path: nessie/src/test/java/org/apache/iceberg/nessie/TestCatalog.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 java.util.List;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCatalog extends BaseTestIceberg {
+
+  private static final String BRANCH = "test-catalog-branch";
+
+  public TestCatalog() {
+    super(BRANCH);
+  }
+
+  @Test
+  public void test() {
+    createTable(TableIdentifier.of("foo", "bar"));
+    List<TableIdentifier> tables = catalog.listTables(Namespace.of("foo"));
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("bar", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.renameTable(TableIdentifier.of("foo", "bar"), TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(null);
+    Assert.assertEquals(1, tables.size());
+    Assert.assertEquals("baz", tables.get(0).name());
+    Assert.assertEquals("foo", tables.get(0).namespace().toString());
+    catalog.dropTable(TableIdentifier.of("foo", "baz"));
+    tables = catalog.listTables(Namespace.empty());
+    Assert.assertTrue(tables.isEmpty());

Review comment:
       I prefer splitting as well. I have split into 3 tests, its a little more cluttered this way but let me know what you think, happy to go either way.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org