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 2023/01/02 09:57:42 UTC

[GitHub] [iceberg] krvikash opened a new pull request, #6512: Core: Support registerTable with REST session catalog

krvikash opened a new pull request, #6512:
URL: https://github.com/apache/iceberg/pull/6512

   Core: Support `registerTable` with REST session 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1230705685


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -400,7 +402,37 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Invalid metadata file location: %s",
+        metadataFileLocation);
+
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    RegisterTableRequest request = new RegisterTableRequest(ident.name(), metadataFileLocation);
+
+    LoadTableResponse response =
+        client.post(
+            paths.register(ident.namespace()),
+            request,
+            LoadTableResponse.class,
+            headers(context),
+            ErrorHandlers.tableErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    RESTTableOperations ops =
+        new RESTTableOperations(
+            client,
+            paths.table(ident),
+            session::headers,
+            tableFileIO(context, response.config()),
+            response.tableMetadata());
+
+    return new BaseTable(ops, fullTableName(ident));

Review Comment:
   just an FYI that we'd need to call `trackFileIO(ops);` just before the return here



##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -400,7 +402,37 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Invalid metadata file location: %s",
+        metadataFileLocation);
+
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    RegisterTableRequest request = new RegisterTableRequest(ident.name(), metadataFileLocation);

Review Comment:
   I think it's better to track the `TableIdentifier` instead of a plain string



##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class RegisterTableRequestParser {
+
+  private static final String TABLE_NAME = "table-name";
+  private static final String METADATA_FILE_LOCATION = "metadata-file-location";
+
+  private RegisterTableRequestParser() {}
+
+  public static String toJson(RegisterTableRequest request) {
+    return toJson(request, false);
+  }
+
+  public static String toJson(RegisterTableRequest request, boolean pretty) {
+    return JsonUtil.generate(gen -> toJson(request, gen), pretty);
+  }
+
+  public static void toJson(RegisterTableRequest request, JsonGenerator gen) throws IOException {
+    Preconditions.checkArgument(null != request, "Invalid register table request: null");
+
+    gen.writeStartObject();
+
+    gen.writeStringField(TABLE_NAME, request.tableName());

Review Comment:
   once we track the `TableIdentifier` in the request, you can change this to 
   ```
   gen.writeFieldName(IDENTIFIER);
   TableIdentifierParser.toJson(request.identifier(), gen);
   ```



##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class RegisterTableRequestParser {
+
+  private static final String TABLE_NAME = "table-name";
+  private static final String METADATA_FILE_LOCATION = "metadata-file-location";
+
+  private RegisterTableRequestParser() {}
+
+  public static String toJson(RegisterTableRequest request) {
+    return toJson(request, false);
+  }
+
+  public static String toJson(RegisterTableRequest request, boolean pretty) {
+    return JsonUtil.generate(gen -> toJson(request, gen), pretty);
+  }
+
+  public static void toJson(RegisterTableRequest request, JsonGenerator gen) throws IOException {
+    Preconditions.checkArgument(null != request, "Invalid register table request: null");
+
+    gen.writeStartObject();
+
+    gen.writeStringField(TABLE_NAME, request.tableName());
+    gen.writeStringField(METADATA_FILE_LOCATION, request.metadataFileLocation());
+
+    gen.writeEndObject();
+  }
+
+  public static RegisterTableRequest fromJson(String json) {
+    return JsonUtil.parse(json, RegisterTableRequestParser::fromJson);
+  }
+
+  public static RegisterTableRequest fromJson(JsonNode json) {
+    Preconditions.checkArgument(
+        null != json, "Cannot parse register table request from null object");
+
+    String name = JsonUtil.getString(TABLE_NAME, json);

Review Comment:
   and this would be `TableIdentifierParser.fromJson(JsonUtil.get(IDENTIFIER, node))`



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1592080049

   Thanks @nastra | @ajantha-bhat for the review. Addressed comments.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235513526


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.catalog.TableIdentifierParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class RegisterTableRequestParser {
+
+  private static final String IDENTIFIER = "identifier";
+  private static final String METADATA_FILE_LOCATION = "metadata-file-location";
+
+  private RegisterTableRequestParser() {}
+
+  public static String toJson(RegisterTableRequest request) {
+    return toJson(request, false);
+  }
+
+  public static String toJson(RegisterTableRequest request, boolean pretty) {
+    return JsonUtil.generate(gen -> toJson(request, gen), pretty);
+  }
+
+  public static void toJson(RegisterTableRequest request, JsonGenerator gen) throws IOException {
+    Preconditions.checkArgument(null != request, "Invalid register table request: null");
+
+    gen.writeStartObject();
+
+    gen.writeFieldName(IDENTIFIER);
+    TableIdentifierParser.toJson(request.identifier(), gen);

Review Comment:
   Why send the full identifier if the namespace is in the request path? Seems like unnecessary duplication to me.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235521123


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,100 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table table =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    table.newFastAppend().appendFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_B).commit();
+    table.newDelete().deleteFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) table).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();

Review Comment:
   I think this should also have a `tableExists` check/assertion just after this point. The catalog can't just return `LoadTableResponse`. It needs to have the table defined.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1236433705


##########
core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java:
##########
@@ -91,8 +94,12 @@ public static void registerAll(ObjectMapper mapper) {
         .addDeserializer(
             ImmutableReportMetricsRequest.class, new ReportMetricsRequestDeserializer<>())
         .addSerializer(CommitTransactionRequest.class, new CommitTransactionRequestSerializer())
+        .addDeserializer(CommitTransactionRequest.class, new CommitTransactionRequestDeserializer())
+        .addSerializer(RegisterTableRequest.class, new RegisterTableRequestSerializer<>())
+        .addDeserializer(RegisterTableRequest.class, new RegisterTableRequestDeserializer<>())

Review Comment:
   we need to register both classes unfortunately. See also https://github.com/apache/iceberg/pull/5407#discussion_r984472975 for some context and  https://github.com/apache/iceberg/pull/5911 where I attempted to fix 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1205769107


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.rest.requests;
+
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.rest.RESTRequest;
+
+public class RegisterTableRequest implements RESTRequest {

Review Comment:
   @rdblue I'm not sure if we decided on which pattern we'd want to use with **new** request/response classes:
   1) class with a public constructor and no explicit JSON parser + manually written builder
   2) Immutable class (similar to `ReportMetricsRequest`) with a separate JSON parser
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1591972748

   (rebased with master)


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1595112002

   (rebased with master and resolved conflicts)


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1610042655

   (resolved conflicts)


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1610111920

   Thanks for reviewing @bryanck, Addressed 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1559849682

   Fixed CI failure.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1066006935


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,30 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Cannot register an empty metadata file location as a table");
+
+    // Throw an exception if this table already exists in the catalog.
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    Map<String, String> tableProperties =
+        ImmutableMap.<String, String>builder()
+            .putAll(metadata.properties())
+            .put(BaseMetastoreTableOperations.METADATA_LOCATION_PROP, metadataFileLocation)
+            .build();
+
+    return buildTable(context, ident, metadata.schema())
+        .withLocation(metadata.location())
+        .withProperties(tableProperties)
+        .withPartitionSpec(metadata.spec())
+        .withSortOrder(metadata.sortOrder())
+        .create();

Review Comment:
   This discards all table data and older schemas, specs, and sort orders. I think this needs to preserve the table metadata completely.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1368972062

   CI failure will be fixed with https://github.com/apache/iceberg/pull/6511


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1595691866

   @nastra, Addressed comments.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] ajantha-bhat commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "ajantha-bhat (via GitHub)" <gi...@apache.org>.
ajantha-bhat commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1229341879


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -119,6 +120,11 @@ enum Route {
         LoadTableResponse.class),
     LOAD_TABLE(
         HTTPMethod.GET, "v1/namespaces/{namespace}/tables/{table}", null, LoadTableResponse.class),
+    REGISTER_TABLE(
+        HTTPMethod.POST,
+        "v1/namespaces/{namespace}/tables/register",
+        RegisterTableRequest.class,
+        LoadTableResponse.class),

Review Comment:
   do we need a new response type? Also, this is not inline with the open-api spec response type.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1244994033


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1913,6 +1965,17 @@ components:
           additionalProperties:
             type: string
 
+    RegisterTableRequest:
+      type: object
+      required:
+        - name
+        - metadata-location
+      properties:
+        identifier:

Review Comment:
   ```suggestion
           name:
   ```



##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -344,7 +350,13 @@ public <T extends RESTResponse> T handleRequest(
           TableIdentifier ident = identFromPathVars(vars);
           return castResponse(responseType, CatalogHandlers.loadTable(catalog, ident));
         }
-
+      case REGISTER_TABLE:
+        {
+          Namespace namespace = namespaceFromPathVars(vars);
+          RegisterTableRequest request = castRequest(RegisterTableRequest.class, body);
+          return castResponse(
+              responseType, CatalogHandlers.registerTable(catalog, namespace, request));
+        }

Review Comment:
   nit: add an empty line after the closing } to align with all the code inside this switch block



##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -344,7 +350,13 @@ public <T extends RESTResponse> T handleRequest(
           TableIdentifier ident = identFromPathVars(vars);
           return castResponse(responseType, CatalogHandlers.loadTable(catalog, ident));
         }
-

Review Comment:
   nit: we probably want to keep that empty line



##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1913,6 +1965,17 @@ components:
           additionalProperties:
             type: string
 
+    RegisterTableRequest:
+      type: object
+      required:
+        - name
+        - metadata-location
+      properties:
+        identifier:

Review Comment:
   this should fix the CI failure. Also make sure to go through https://github.com/apache/iceberg/blob/master/open-api/README.md#generate-python-code



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1251275726


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,95 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table originalTable =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    originalTable.newFastAppend().appendFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_B).commit();
+    originalTable.newDelete().deleteFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) originalTable).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+    Assertions.assertThat(catalog.tableExists(TABLE)).as("Table must exist").isTrue();
+
+    TableOperations registeredTableOps = ((HasTableOperations) originalTable).operations();
+    Assertions.assertThat(registeredTableOps.current().metadataFileLocation())
+        .isEqualTo(metadataLocation);
+
+    Assertions.assertThat(registeredTable.properties())

Review Comment:
   This needs an assertion for the table UUID.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1134061896


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -391,7 +393,41 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Invalid metadata file location: %s",
+        metadataFileLocation);
+
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    RegisterTableRequest request =
+        RegisterTableRequest.builder()
+            .withName(ident.name())
+            .withMetadataFielLocation(metadataFileLocation)

Review Comment:
   typo in `withMetadataFielLocation` -> should be `withMetadataFileLocation`



##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -335,7 +341,14 @@ public <T extends RESTResponse> T handleRequest(
           TableIdentifier ident = identFromPathVars(vars);
           return castResponse(responseType, CatalogHandlers.loadTable(catalog, ident));
         }
-
+      case REGISTER_TABLE:
+        {
+          Namespace namespace = namespaceFromPathVars(vars);
+          RegisterTableRequest request = castRequest(RegisterTableRequest.class, body);
+          request.validate();

Review Comment:
   I think this isn't necessary to do here as it's already being done in `CatalogHandlers.registerTable`



##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -496,6 +496,58 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/namespaces/{namespace}/tables/register:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Create a table in the given namespace using given metadata file location

Review Comment:
   Create a table -> Register 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1610039307

   (Modified `RegisterTableRequest` methods)


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1065484548


##########
core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java:
##########
@@ -146,6 +149,21 @@ public void doCommit(TableMetadata base, TableMetadata metadata) {
     }
   }
 
+  private Optional<String> getExistingMetadataLocation(TableMetadata base, TableMetadata metadata) {

Review Comment:
   Thanks, @nastra for your review. I will create a new PR for this part of the change.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1065101723


##########
core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java:
##########
@@ -146,6 +149,21 @@ public void doCommit(TableMetadata base, TableMetadata metadata) {
     }
   }
 
+  private Optional<String> getExistingMetadataLocation(TableMetadata base, TableMetadata metadata) {

Review Comment:
   Yes, You are right. without this change also register table will work. But the `registerTable` will create a new metadata JSON file and register the table. That is why this handling ensures if `registerTable` call `doCommit` then don't create a new metadata file and rather use the existing one.
   
   Currently we are handling the same logic in `HiveTableOperations` and `NessieTableOperations`. But other table operations are missing this part.
   
   https://github.com/apache/iceberg/blob/6b8f7e0e31a81029b478e7757aba749f5ed27f42/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java#L249-L252
   
   https://github.com/apache/iceberg/blob/ede085d0f7529f24acd0c81dd0a43f7bb969b763/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java#L146-L149
   
   
    
   Coming to `getExistingMetadataLocation`, I am setting `metadataFileLocation` in the meadata properties in `RESTSessionCatalog` whereas for `JdbcCatalog` it uses `BaseMetastoreCatalog` implementation where `metadata` with existing `metadataFileLocation` is passed through 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1066004205


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,30 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Cannot register an empty metadata file location as a table");
+
+    // Throw an exception if this table already exists in the catalog.

Review Comment:
   This comment is obvious from context, can you please remove 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1620742492

   Thanks, @krvikash! Great to have this supported in the REST 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1593495120

   rebased with master and fixed CI failure.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1593474820

   Thank you @nastra for the review. Addressed comments.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1231898096


##########
core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java:
##########
@@ -61,6 +61,11 @@ public String tables(Namespace ns) {
     return SLASH.join("v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "tables");
   }
 
+  public String register(Namespace ns) {
+    return SLASH.join(
+        "v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "tables", "register");

Review Comment:
   we've had some internal discussion on what the pathing should look like and `/v1/{prefix}/namespaces/{namespace}/register` makes probably the most sense here. 
   Having `/v1/{prefix}/namespaces/{namespace}/tables/register` would conflict when a table is called `register`



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1230238829


##########
core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java:
##########
@@ -146,6 +149,21 @@ public void doCommit(TableMetadata base, TableMetadata metadata) {
     }
   }
 
+  private Optional<String> getExistingMetadataLocation(TableMetadata base, TableMetadata metadata) {

Review Comment:
   https://github.com/apache/iceberg/pull/6591



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235919848


##########
core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java:
##########
@@ -91,8 +94,12 @@ public static void registerAll(ObjectMapper mapper) {
         .addDeserializer(
             ImmutableReportMetricsRequest.class, new ReportMetricsRequestDeserializer<>())
         .addSerializer(CommitTransactionRequest.class, new CommitTransactionRequestSerializer())
+        .addDeserializer(CommitTransactionRequest.class, new CommitTransactionRequestDeserializer())
+        .addSerializer(RegisterTableRequest.class, new RegisterTableRequestSerializer<>())
+        .addDeserializer(RegisterTableRequest.class, new RegisterTableRequestDeserializer<>())

Review Comment:
   Removing any one of them throws the below response. I see `ReportMetricsRequest` register both the interface and the implementation.
   
   ```
       org.apache.iceberg.exceptions.ServiceFailureException: Server error: null: {
           "servlet":"org.apache.iceberg.rest.RESTCatalogServlet-4c38cd16",
           "cause0":"com.fasterxml.jackson.databind.exc.InvalidDefinitionException: Cannot construct instance of `org.apache.iceberg.rest.requests.RegisterTableRequest` (no Creators, like default constructor, exist): abstract types either need to be mapped to concrete types, have custom deserializer, or contain additional type information\n at [Source: (org.eclipse.jetty.server.Request$1); line: 1, column: 1]",
           "message":"com.fasterxml.jackson.databind.exc.InvalidDefinitionException: Cannot construct instance of `org.apache.iceberg.rest.requests.RegisterTableRequest` (no Creators, like default constructor, exist): abstract types either need to be mapped to concrete types, have custom deserializer, or contain additional type information\n at [Source: (org.eclipse.jetty.server.Request$1); line: 1, column: 1]",
           "url":"/v1/namespaces/newdb/register",
           "status":"500"
           }
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1559812591

   rebased with master and resolved conflicts.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235762784


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,100 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table table =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    table.newFastAppend().appendFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_B).commit();
+    table.newDelete().deleteFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) table).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+
+    TableOperations registeredTableOps = ((HasTableOperations) table).operations();
+    Assertions.assertThat(metadataLocation)
+        .isEqualTo(registeredTableOps.current().metadataFileLocation());

Review Comment:
   Hi @rdblue, https://github.com/apache/iceberg/pull/6591 enforce that the `register_table` avoid creating new metadata and uses the original metadata to register 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235918369


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.catalog.TableIdentifierParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class RegisterTableRequestParser {
+
+  private static final String IDENTIFIER = "identifier";
+  private static final String METADATA_FILE_LOCATION = "metadata-file-location";
+
+  private RegisterTableRequestParser() {}
+
+  public static String toJson(RegisterTableRequest request) {
+    return toJson(request, false);
+  }
+
+  public static String toJson(RegisterTableRequest request, boolean pretty) {
+    return JsonUtil.generate(gen -> toJson(request, gen), pretty);
+  }
+
+  public static void toJson(RegisterTableRequest request, JsonGenerator gen) throws IOException {
+    Preconditions.checkArgument(null != request, "Invalid register table request: null");
+
+    gen.writeStartObject();
+
+    gen.writeFieldName(IDENTIFIER);
+    TableIdentifierParser.toJson(request.identifier(), gen);

Review Comment:
   As @nastra suggested (https://github.com/apache/iceberg/pull/6512#discussion_r1230706545 and  https://github.com/apache/iceberg/pull/6512#discussion_r1230708833). I modified it to send identifier.
   > I think it's better to track the TableIdentifier instead of a plain string
   
   I see that `UpdateTableRequestParser` is doing the same thing. 
   
   https://github.com/apache/iceberg/blob/9ffb762153ce050c2d961327886ef9a41b2d6145/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequestParser.java#L56-L57



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] ajantha-bhat commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "ajantha-bhat (via GitHub)" <gi...@apache.org>.
ajantha-bhat commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1229315482


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -496,6 +496,58 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/namespaces/{namespace}/tables/register:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Register a table in the given namespace using given metadata file location
+      description:
+        Register a table using given metadata file location.
+
+      operationId: registerTable
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/RegisterTableRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/CreateTableResponse'

Review Comment:
   should this be `RegisterTableResponse`?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] bryanck commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "bryanck (via GitHub)" <gi...@apache.org>.
bryanck commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1244210182


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.rest.requests;
+
+import org.apache.iceberg.rest.RESTRequest;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface RegisterTableRequest extends RESTRequest {
+
+  String tableName();

Review Comment:
   I feel this should be named `name` to align with `CreateTableRequest`.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1244202680


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.catalog.TableIdentifierParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class RegisterTableRequestParser {
+
+  private static final String IDENTIFIER = "identifier";
+  private static final String METADATA_FILE_LOCATION = "metadata-file-location";
+
+  private RegisterTableRequestParser() {}
+
+  public static String toJson(RegisterTableRequest request) {
+    return toJson(request, false);
+  }
+
+  public static String toJson(RegisterTableRequest request, boolean pretty) {
+    return JsonUtil.generate(gen -> toJson(request, gen), pretty);
+  }
+
+  public static void toJson(RegisterTableRequest request, JsonGenerator gen) throws IOException {
+    Preconditions.checkArgument(null != request, "Invalid register table request: null");
+
+    gen.writeStartObject();
+
+    gen.writeFieldName(IDENTIFIER);
+    TableIdentifierParser.toJson(request.identifier(), gen);

Review Comment:
   Hi @rdblue | @nastra, I have changed the `RegisterTableRequest` to have table-name instead identifier.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1251274006


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,95 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table originalTable =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    originalTable.newFastAppend().appendFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_B).commit();
+    originalTable.newDelete().deleteFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) originalTable).operations();

Review Comment:
   Use `BaseTable` instead of `HasTableOperations`.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] bryanck commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "bryanck (via GitHub)" <gi...@apache.org>.
bryanck commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1242632371


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.rest.requests;
+
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.rest.RESTRequest;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface RegisterTableRequest extends RESTRequest {
+
+  TableIdentifier identifier();

Review Comment:
   The namespace in the `TableIdentifier` seems redundant with the namespace in the path, should this just be table name?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1439095868

   Hi @ajantha-bhat | @nastra | @rdblue, when you get time, could you please review it 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1466262308

   @krvikash looks like CI is failing on this 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] ajantha-bhat commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1064786716


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,35 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override

Review Comment:
   Just wondering why can't we remove this method and let it use the parent implementation.
   Other catalogs do the same. 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1065429572


##########
core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java:
##########
@@ -146,6 +149,21 @@ public void doCommit(TableMetadata base, TableMetadata metadata) {
     }
   }
 
+  private Optional<String> getExistingMetadataLocation(TableMetadata base, TableMetadata metadata) {

Review Comment:
   I would probably do these changes independently from this PR as the scope of this PR should be only around supporting table registration for the REST 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] ajantha-bhat commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "ajantha-bhat (via GitHub)" <gi...@apache.org>.
ajantha-bhat commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1229357563


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -496,6 +496,58 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/namespaces/{namespace}/tables/register:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Register a table in the given namespace using given metadata file location
+      description:
+        Register a table using given metadata file location.
+
+      operationId: registerTable
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/RegisterTableRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/CreateTableResponse'

Review Comment:
   I think this should be `LoadTableResponse` as per the code 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] ajantha-bhat commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "ajantha-bhat (via GitHub)" <gi...@apache.org>.
ajantha-bhat commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1229341879


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -119,6 +120,11 @@ enum Route {
         LoadTableResponse.class),
     LOAD_TABLE(
         HTTPMethod.GET, "v1/namespaces/{namespace}/tables/{table}", null, LoadTableResponse.class),
+    REGISTER_TABLE(
+        HTTPMethod.POST,
+        "v1/namespaces/{namespace}/tables/register",
+        RegisterTableRequest.class,
+        LoadTableResponse.class),

Review Comment:
   do we need a new response type? Also, this is not inline with the open-api spec response type.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235512841


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.catalog.TableIdentifierParser;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class RegisterTableRequestParser {
+
+  private static final String IDENTIFIER = "identifier";
+  private static final String METADATA_FILE_LOCATION = "metadata-file-location";

Review Comment:
   For `LoadTableResponse` this is `metadata-location`. I think this should be the same.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235516238


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,100 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table table =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    table.newFastAppend().appendFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_B).commit();
+    table.newDelete().deleteFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) table).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+
+    TableOperations registeredTableOps = ((HasTableOperations) table).operations();
+    Assertions.assertThat(metadataLocation)
+        .isEqualTo(registeredTableOps.current().metadataFileLocation());

Review Comment:
   This isn't necessarily true. It's fine for the catalog to import and create its own metadata file copy. I think what we want to validate is that the table is identical, not that it uses the original metadata 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235523363


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1913,6 +1965,17 @@ components:
           additionalProperties:
             type: string
 
+    RegisterTableRequest:
+      type: object
+      required:
+        - identifier
+        - metadata-file-location

Review Comment:
   This should be `metadata-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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1251273788


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,100 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table table =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    table.newFastAppend().appendFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_B).commit();
+    table.newDelete().deleteFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) table).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+
+    TableOperations registeredTableOps = ((HasTableOperations) table).operations();
+    Assertions.assertThat(metadataLocation)
+        .isEqualTo(registeredTableOps.current().metadataFileLocation());

Review Comment:
   I don't think that register table should be required to use the same metadata file. Some metadata may be overridden so there's no guarantee it's the same file.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1251275140


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,95 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table originalTable =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    originalTable.newFastAppend().appendFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_B).commit();
+    originalTable.newDelete().deleteFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) originalTable).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+    Assertions.assertThat(catalog.tableExists(TABLE)).as("Table must exist").isTrue();
+
+    TableOperations registeredTableOps = ((HasTableOperations) originalTable).operations();
+    Assertions.assertThat(registeredTableOps.current().metadataFileLocation())
+        .isEqualTo(metadataLocation);

Review Comment:
   I mentioned this elsewhere, but this check is unnecessary.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1251274935


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,95 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table originalTable =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    originalTable.newFastAppend().appendFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_B).commit();
+    originalTable.newDelete().deleteFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) originalTable).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+    Assertions.assertThat(catalog.tableExists(TABLE)).as("Table must exist").isTrue();
+
+    TableOperations registeredTableOps = ((HasTableOperations) originalTable).operations();

Review Comment:
   Use `BaseTable` here as well.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1065103623


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,35 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    Preconditions.checkArgument(
+        ident != null && isValidIdentifier(ident), "Invalid identifier: %s", ident);
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Cannot register an empty metadata file location as a table");
+
+    // Throw an exception if this table already exists in the catalog.
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    Map<String, String> tableProperties =
+        ImmutableMap.<String, String>builder()
+            .putAll(metadata.properties())
+            .put(BaseMetastoreTableOperations.METADATA_LOCATION_PROP, metadataFileLocation)
+            .build();
+
+    return buildTable(context, ident, metadata.schema())
+        .withLocation(metadata.location())
+        .withProperties(tableProperties)
+        .withPartitionSpec(metadata.spec())
+        .withSortOrder(metadata.sortOrder())
+        .create();
+  }
+
+  private boolean isValidIdentifier(TableIdentifier tableIdentifier) {

Review Comment:
   Yes, I have copied `isValidIdentifier` from `BaseMetastoreCatalog` implementation. Thanks for the input. I will use `checkIdentifierIsValid` over 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1064754792


##########
core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java:
##########
@@ -146,6 +149,21 @@ public void doCommit(TableMetadata base, TableMetadata metadata) {
     }
   }
 
+  private Optional<String> getExistingMetadataLocation(TableMetadata base, TableMetadata metadata) {

Review Comment:
   can you elaborate why these changes are needed in order to support registering a table in the REST catalog? I think stuff should work fine without these changes 



##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,35 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    Preconditions.checkArgument(
+        ident != null && isValidIdentifier(ident), "Invalid identifier: %s", ident);
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Cannot register an empty metadata file location as a table");
+
+    // Throw an exception if this table already exists in the catalog.
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation);
+    Map<String, String> tableProperties =
+        ImmutableMap.<String, String>builder()
+            .putAll(metadata.properties())
+            .put(BaseMetastoreTableOperations.METADATA_LOCATION_PROP, metadataFileLocation)
+            .build();
+
+    return buildTable(context, ident, metadata.schema())
+        .withLocation(metadata.location())
+        .withProperties(tableProperties)
+        .withPartitionSpec(metadata.spec())
+        .withSortOrder(metadata.sortOrder())
+        .create();
+  }
+
+  private boolean isValidIdentifier(TableIdentifier tableIdentifier) {

Review Comment:
   do we need this, given that it always returns true? I know we do the same in `BaseMetastoreCatalog` where catalog implementations can override this behavior but I don't think we need this here in particular. 
   You might rather want to call `checkIdentifierIsValid(..)` in this case.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1620985877

   Thanks @rdblue for merging. Thanks @nastra, @ajantha-bhat, @bryanck, @rdblue for the review.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] ajantha-bhat commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "ajantha-bhat (via GitHub)" <gi...@apache.org>.
ajantha-bhat commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1229343486


##########
core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java:
##########
@@ -761,37 +759,6 @@ public void testConversions() {
     Assert.assertEquals(ns, JdbcUtil.stringToNamespace(nsString));
   }
 
-  @Test
-  public void testRegisterTable() {

Review Comment:
   why this is 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1229301817


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.rest.requests;
+
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.rest.RESTRequest;
+
+public class RegisterTableRequest implements RESTRequest {

Review Comment:
   @krvikash since we don't want to rely on reflection (anymore) and want to be explicit, could you please add a separate JSON parser for that request class? This would look similar to `CommitTransactionRequestParser`



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1593475969

   (squashed 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1595751878

   (fixed CI failure)


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1232963122


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -400,7 +403,43 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Invalid metadata file location: %s",
+        metadataFileLocation);
+
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    RegisterTableRequest request =
+        ImmutableRegisterTableRequest.builder()
+            .identifier(ident)
+            .metadataFileLocation(metadataFileLocation)
+            .build();
+
+    LoadTableResponse response =
+        client.post(
+            paths.register(ident.namespace()),
+            request,
+            LoadTableResponse.class,
+            headers(context),
+            ErrorHandlers.tableErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    RESTTableOperations ops =
+        new RESTTableOperations(
+            client,
+            paths.table(ident),
+            session::headers,
+            tableFileIO(context, response.config()),
+            response.tableMetadata());
+
+    trackFileIO(ops);
+
+    return new BaseTable(ops, fullTableName(ident));

Review Comment:
   this also needs a parameter at the end to have proper metrics reporting: `metricsReporter(paths.metrics(ident), session::headers)`



##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -498,6 +498,58 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/namespaces/{namespace}/tables/register:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Register a table in the given namespace using given metadata file location
+      description:
+        Register a table using given metadata file location.
+
+      operationId: registerTable
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/RegisterTableRequest'

Review Comment:
   we also need to define the schema for `RegisterTableRequest`



##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -498,6 +498,58 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/namespaces/{namespace}/tables/register:

Review Comment:
   this seems to still have the old 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1244206041


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,100 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table table =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    table.newFastAppend().appendFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_B).commit();
+    table.newDelete().deleteFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) table).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+
+    TableOperations registeredTableOps = ((HasTableOperations) table).operations();
+    Assertions.assertThat(metadataLocation)
+        .isEqualTo(registeredTableOps.current().metadataFileLocation());

Review Comment:
   Hi @rdblue, what is your thought here?
   
   I see in https://github.com/apache/iceberg/pull/6742, It was required to `registerTable API to avoid creating new metadata file`.
   
   > remove the commit part in registerTable API to avoid creating new metadata file
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1611878106

   Thanks @nastra for the review. Addressed comments.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1559811192

   Addressed comments


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1205770227


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2487,6 +2488,88 @@ public void tableCreationWithoutNamespace() {
         .hasMessageEndingWith("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table table =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    table.newFastAppend().appendFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_B).commit();
+    table.newDelete().deleteFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) table).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+
+    TableOperations registeredTableOps = ((HasTableOperations) table).operations();
+    Assertions.assertThat(metadataLocation)
+        .isEqualTo(registeredTableOps.current().metadataFileLocation());
+
+    Map<String, String> actualProperties = Maps.newHashMap(table.properties());
+    Map<String, String> expectedProperties = Maps.newHashMap(table.properties());
+    // The nessie.commit.id for each table will be distinct, hence, it is necessary to remove it
+    // before asserting the properties.
+    actualProperties.remove("nessie.commit.id");
+    expectedProperties.remove("nessie.commit.id");
+
+    Assert.assertEquals("Location must match", table.location(), registeredTable.location());
+    Assert.assertEquals("Props must match", actualProperties, expectedProperties);
+    Assert.assertEquals(
+        "Schema must match", table.schema().asStruct(), registeredTable.schema().asStruct());
+    Assert.assertEquals("Specs must match", table.specs(), registeredTable.specs());
+    Assert.assertEquals("Sort orders must match", table.sortOrders(), registeredTable.sortOrders());
+    Assert.assertEquals(
+        "Current snapshot must match", table.currentSnapshot(), registeredTable.currentSnapshot());
+    Assert.assertEquals("Snapshots must match", table.snapshots(), registeredTable.snapshots());
+    Assert.assertEquals("History must match", table.history(), registeredTable.history());
+
+    TestHelpers.assertSameSchemaMap(table.schemas(), registeredTable.schemas());
+    assertFiles(registeredTable, FILE_B, FILE_C);
+
+    registeredTable.newFastAppend().appendFile(FILE_A).commit();
+    assertFiles(registeredTable, FILE_B, FILE_C, FILE_A);
+
+    Assertions.assertThat(catalog.loadTable(TABLE)).isNotNull();
+    Assertions.assertThat(catalog.dropTable(TABLE)).isTrue();
+    Assertions.assertThat(catalog.tableExists(TABLE)).isFalse();
+  }
+
+  @Test
+  public void testRegisterExistingTable() {
+    C catalog = catalog();
+
+    TableIdentifier identifier = TableIdentifier.of("a", "t1");
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(identifier.namespace());
+    }
+    catalog.createTable(identifier, SCHEMA);

Review Comment:
   nit: space missing after }



##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.rest.requests;
+
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.rest.RESTRequest;
+
+public class RegisterTableRequest implements RESTRequest {

Review Comment:
   @rdblue I'm not sure if we decided on which pattern we'd want to use with new request/response classes:
   1) class with a public constructor and no explicit JSON parser + manually written builder
   2) Immutable class (similar to `ReportMetricsRequest`) with a separate JSON parser
   



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] ajantha-bhat commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1064792238


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,35 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override

Review Comment:
   oh.. This is `SessionCatalog`, not the `Catalog`. So, there is no parent implementation. 
   I think this logic can be moved to `BaseSessionCatalog`. So, it can help if there are some more implementations in the future. 



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1473182039

   Thanks, @nastra for the reminder. I have fixed the CI failure.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235522976


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -400,7 +403,44 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Invalid metadata file location: %s",
+        metadataFileLocation);
+
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    RegisterTableRequest request =
+        ImmutableRegisterTableRequest.builder()
+            .identifier(ident)
+            .metadataFileLocation(metadataFileLocation)
+            .build();
+
+    LoadTableResponse response =
+        client.post(
+            paths.register(ident.namespace()),
+            request,
+            LoadTableResponse.class,
+            headers(context),
+            ErrorHandlers.tableErrorHandler());

Review Comment:
   The error handler looks correct. 409 results in `AlreadyExistsException`



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235506440


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -400,7 +403,44 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Invalid metadata file location: %s",
+        metadataFileLocation);
+
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);

Review Comment:
   I think we should allow the catalog to return this instead of checking with a second request.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1599142669

   @krvikash, this is looking great! I noted a few things to fix, but overall I think the implementation is solid. Thanks for working on 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1244201531


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.rest.requests;
+
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.rest.RESTRequest;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface RegisterTableRequest extends RESTRequest {
+
+  TableIdentifier identifier();

Review Comment:
   Thanks @bryanck for the review. I have updated the code to take just table-name in the request.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1065104918


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,35 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override

Review Comment:
   Thanks, @ajantha-bhat for the input. Yes, moving implementation logic makes sense to `BaseSessionCatalog`. I will update the code.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1251274599


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,95 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table originalTable =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    originalTable.newFastAppend().appendFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_B).commit();
+    originalTable.newDelete().deleteFile(FILE_A).commit();
+    originalTable.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) originalTable).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);

Review Comment:
   When passing a boolean, always use a comment to clarify what it means:
   
   ```java
       catalog.dropTable(TABLE, false /* do not purge */);
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1595033835

   Thanks @nastra, Addressed 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue merged PR #6512:
URL: https://github.com/apache/iceberg/pull/6512


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1382956003

   CI Failure will be fixed by https://github.com/apache/iceberg/pull/6589
   
   ```
   TestJdbcCatalog > testRegisterTable() FAILED
       org.junit.ComparisonFailure: Name must match expected:<[test_jdbc_catalog.]newdb.table> but was:<[]newdb.table>
           at org.junit.Assert.assertEquals(Assert.java:117)
   ```
   
   ```
   TestNessieCatalog > testRegisterTable() FAILED
       org.junit.ComparisonFailure: Name must match expected:<ne[ssie.ne]wdb.table> but was:<ne[]wdb.table>
           at org.junit.Assert.assertEquals(Assert.java:117)
   ```


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1382904647

   > If we want to support register, then I think the protocol itself should support register
   
   Hi @rdblue, Thanks for the review. If I understood correctly, you wanted to introduce a new `RESTRequest` for `registerTable` similar to what we have `CreateTableRequest`. I have introduced a new `RegisterTableRequest` and used it for `regsiterTable`. I have updated the test case as well where I am verifying verify metadata like specs, sortOrders, history, snapshots ...
   
   Please let me know if this is not what you intended to convey.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1065111504


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,35 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override

Review Comment:
   I see that all the method's implementation is in `RESTSessionCatalog.java`. IMO we should keep `registerTable` implementation in `RESTSessionCatalog.java` as well. WDYT?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1066003697


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -351,7 +354,30 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Cannot register an empty metadata file location as a table");

Review Comment:
   If the metadata location is null, this should not state that it was empty. Instead, use the more standard form for error messages: `"Invalid metadata file location: %s"`. Printing the location shows that it is null or empty.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1508132536

   Hi @rdblue | @nastra | @ajantha-bhat , When you get time, could you please review 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] ajantha-bhat commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "ajantha-bhat (via GitHub)" <gi...@apache.org>.
ajantha-bhat commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1229315482


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -496,6 +496,58 @@ paths:
         5XX:
           $ref: '#/components/responses/ServerErrorResponse'
 
+  /v1/{prefix}/namespaces/{namespace}/tables/register:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Register a table in the given namespace using given metadata file location
+      description:
+        Register a table using given metadata file location.
+
+      operationId: registerTable
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/RegisterTableRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/CreateTableResponse'

Review Comment:
   should this be `RegisterTableResponse`?



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1619476514

   @rdblue, thank you for reviewing. I have addressed all comments.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1230694022


##########
core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.rest.requests;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class RegisterTableRequestParser {
+
+  private static final String TABLE_NAME = "table-name";

Review Comment:
   I believe in other places we're using `identifier` instead of table name, so might be better to use the same here as well



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on a diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1202701352


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -391,7 +393,41 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {}
   @Override
   public Table registerTable(
       SessionContext context, TableIdentifier ident, String metadataFileLocation) {
-    throw new UnsupportedOperationException("Register table is not supported");
+    checkIdentifierIsValid(ident);
+
+    Preconditions.checkArgument(
+        metadataFileLocation != null && !metadataFileLocation.isEmpty(),
+        "Invalid metadata file location: %s",
+        metadataFileLocation);
+
+    if (tableExists(context, ident)) {
+      throw new AlreadyExistsException("Table already exists: %s", ident);
+    }
+
+    RegisterTableRequest request =
+        RegisterTableRequest.builder()
+            .withName(ident.name())
+            .withMetadataFielLocation(metadataFileLocation)

Review Comment:
   Thanks @nastra for catching 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235503888


##########
core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java:
##########
@@ -91,8 +94,12 @@ public static void registerAll(ObjectMapper mapper) {
         .addDeserializer(
             ImmutableReportMetricsRequest.class, new ReportMetricsRequestDeserializer<>())
         .addSerializer(CommitTransactionRequest.class, new CommitTransactionRequestSerializer())
+        .addDeserializer(CommitTransactionRequest.class, new CommitTransactionRequestDeserializer())
+        .addSerializer(RegisterTableRequest.class, new RegisterTableRequestSerializer<>())
+        .addDeserializer(RegisterTableRequest.class, new RegisterTableRequestDeserializer<>())

Review Comment:
   Why do we register both the interface and the implementation? Shouldn't we just need 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235519344


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,100 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table table =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    table.newFastAppend().appendFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_B).commit();
+    table.newDelete().deleteFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) table).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+
+    TableOperations registeredTableOps = ((HasTableOperations) table).operations();
+    Assertions.assertThat(metadataLocation)
+        .isEqualTo(registeredTableOps.current().metadataFileLocation());
+
+    Map<String, String> actualProperties = Maps.newHashMap(table.properties());
+    Map<String, String> expectedProperties = Maps.newHashMap(table.properties());
+    // The nessie.commit.id for each table will be distinct, hence, it is necessary to remove it
+    // before asserting the properties.
+    actualProperties.remove("nessie.commit.id");
+    expectedProperties.remove("nessie.commit.id");
+
+    Assertions.assertThat(table.location())
+        .as("Location must match")
+        .isEqualTo(registeredTable.location());

Review Comment:
   I think this can be overridden by the catalog, I'd omit this check.



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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 diff in pull request #6512: Core: Support registerTable with REST session catalog

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#discussion_r1235518913


##########
core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java:
##########
@@ -2615,6 +2616,100 @@ public void tableCreationWithoutNamespace() {
         .hasMessageContaining("Namespace does not exist: non-existing");
   }
 
+  @Test
+  public void testRegisterTable() {
+    C catalog = catalog();
+
+    if (requiresNamespaceCreate()) {
+      catalog.createNamespace(TABLE.namespace());
+    }
+
+    Map<String, String> properties =
+        ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01");
+    Table table =
+        catalog
+            .buildTable(TABLE, SCHEMA)
+            .withPartitionSpec(SPEC)
+            .withSortOrder(WRITE_ORDER)
+            .withProperties(properties)
+            .create();
+
+    table.newFastAppend().appendFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_B).commit();
+    table.newDelete().deleteFile(FILE_A).commit();
+    table.newFastAppend().appendFile(FILE_C).commit();
+
+    TableOperations ops = ((HasTableOperations) table).operations();
+    String metadataLocation = ops.current().metadataFileLocation();
+
+    catalog.dropTable(TABLE, false);
+
+    Table registeredTable = catalog.registerTable(TABLE, metadataLocation);
+    Assertions.assertThat(registeredTable).isNotNull();
+
+    TableOperations registeredTableOps = ((HasTableOperations) table).operations();
+    Assertions.assertThat(metadataLocation)
+        .isEqualTo(registeredTableOps.current().metadataFileLocation());
+
+    Map<String, String> actualProperties = Maps.newHashMap(table.properties());
+    Map<String, String> expectedProperties = Maps.newHashMap(table.properties());
+    // The nessie.commit.id for each table will be distinct, hence, it is necessary to remove it
+    // before asserting the properties.

Review Comment:
   `CatalogTests` should not have anything specific to Nessie. Instead of doing this, I think the validation should check that the properties used to create the table are a subset of the properties in the actual 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1599611754

   Thanks @rdblue for the review, I have addressed the comments. However, there are a few open conversations.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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] krvikash commented on pull request #6512: Core: Support registerTable with REST session catalog

Posted by "krvikash (via GitHub)" <gi...@apache.org>.
krvikash commented on PR #6512:
URL: https://github.com/apache/iceberg/pull/6512#issuecomment-1439094408

   Rebased with master and resolved conflicts.


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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