You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "nastra (via GitHub)" <gi...@apache.org> on 2023/06/26 13:12:46 UTC

[GitHub] [iceberg] nastra opened a new pull request, #7913: Core: Add View support for REST catalog

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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -30,8 +30,10 @@
 import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.view.View;
+import org.apache.iceberg.view.ViewBuilder;
 
-public abstract class BaseSessionCatalog implements SessionCatalog {
+public abstract class BaseSessionCatalog implements SessionCatalog, ViewSessionCatalog {

Review Comment:
   in this case, maybe we should have a `BaseViewSessionCatalog` that extends `BaseSessionCatalog`? This would be similar to `BaseMetastoreCatalog` & `BaseMetastoreViewCatalog`.
   Also I'm not sure why RevAPI didn't complain



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1014,6 +1014,357 @@ paths:
                 }
               }
 
+  /v1/{prefix}/namespaces/{namespace}/views:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    get:
+      tags:
+        - Catalog API
+      summary: List all view identifiers underneath a given namespace
+      description: Return all view identifiers under this namespace
+      operationId: listViews
+      responses:
+        200:
+          $ref: '#/components/responses/ListTablesResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Create a view in the given namespace
+      description:
+        Create a view in the given namespace.
+      operationId: createView
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CreateViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        409:
+          description: Conflict - The view already exists
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceAlreadyExists:
+                  $ref: '#/components/examples/ViewAlreadyExistsError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/namespaces/{namespace}/views/{view}:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+      - $ref: '#/components/parameters/view'
+
+    get:
+      tags:
+        - Catalog API
+      summary: Load a view from the catalog
+      operationId: loadView
+      description:
+        Load a view from the catalog.
+
+
+        The response contains both configuration and table metadata. The configuration, if non-empty is used
+        as additional configuration for the view that overrides catalog configuration. For example, this
+        configuration may change the FileIO implementation to be used for the view.
+
+
+        The response also contains the view's full metadata, matching the view metadata JSON file.
+
+
+        The catalog configuration may contain credentials that should be used for subsequent requests for the
+        view. The configuration key "token" is used to pass an access token to be used as a bearer token
+        for view requests. Otherwise, a token may be passed using a RFC 8693 token type as a configuration
+        key. For example, "urn:ietf:params:oauth:token-type:jwt=<JWT-token>".
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToLoadDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Replace a view
+      operationId: replaceView
+      description:
+        Commit updates to a view.
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToUpdateDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        409:
+          description:
+            Conflict - CommitFailedException. The client may retry.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        500:
+          description:
+            An unknown server-side problem occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Internal Server Error",
+                  "type": "CommitStateUnknownException",
+                  "code": 500
+                }
+              }
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        502:
+          description:
+            A gateway or proxy received an invalid response from the upstream server; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Invalid response from the upstream server",
+                  "type": "CommitStateUnknownException",
+                  "code": 502
+                }
+              }
+        504:
+          description:
+            A server-side gateway timeout occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Gateway timed out during commit",
+                  "type": "CommitStateUnknownException",
+                  "code": 504
+                }
+              }
+        5XX:
+          description:
+            A server-side problem that might not be addressable on the client.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Bad Gateway",
+                  "type": "InternalServerError",
+                  "code": 502
+                }
+              }
+
+    delete:
+      tags:
+        - Catalog API
+      summary: Drop a view from the catalog
+      operationId: dropView
+      description: Remove a view from the catalog
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to drop does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToDeleteDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    head:
+      tags:
+        - Catalog API
+      summary: Check if a view exists
+      operationId: viewExists
+      description:
+        Check if a view exists within a given namespace. This request does not return a response body.
+      responses:
+        200:
+          description: OK - View Exists
+        400:
+          description: Bad Request
+        401:
+          description: Unauthorized
+        404:
+          description: Not Found
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/views/rename:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Rename a view from its current name to a new name
+      description:
+        Rename a view from one identifier to another. It's valid to move a view
+        across namespaces, but the server implementation is not required to support it.
+      operationId: renameView
+      requestBody:
+        description: Current view identifier to rename and new view identifier to rename to
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/RenameTableRequest'
+            examples:
+              RenameViewSameNamespace:
+                $ref: '#/components/examples/RenameViewSameNamespace'
+        required: true
+      responses:
+        200:
+          description: OK
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found
+            - NoSuchViewException, view to rename does not exist
+            - NoSuchNamespaceException, The target namespace of the new table identifier does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToRenameDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+                NamespaceToRenameToDoesNotExist:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        406:
+          $ref: '#/components/responses/UnsupportedOperationResponse'
+        409:
+          description: Conflict - The target view identifier to rename to already exists

Review Comment:
   Do we have test cases for rename when the view exists as a table? Or similarly, rename table when the view exists?



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest;
+import org.apache.iceberg.rest.responses.LoadViewResponse;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.iceberg.view.ViewOperations;
+
+class RESTViewOperations implements ViewOperations {
+
+  private final RESTClient client;
+  private final String path;
+  private final Supplier<Map<String, String>> headers;
+  private final FileIO io;
+  private ViewMetadata current;
+
+  RESTViewOperations(
+      RESTClient client,
+      String path,
+      Supplier<Map<String, String>> headers,
+      FileIO io,
+      ViewMetadata current) {
+    Preconditions.checkArgument(null != current, "Invalid view metadata: null");
+    this.client = client;
+    this.path = path;
+    this.headers = headers;
+    this.io = io;
+    this.current = current;
+  }
+
+  @Override
+  public ViewMetadata current() {
+    return current;
+  }
+
+  @Override
+  public ViewMetadata refresh() {
+    return updateCurrentMetadata(
+        client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler()));
+  }
+
+  @Override
+  public void commit(ViewMetadata base, ViewMetadata metadata) {
+    // this is only used for replacing view metadata
+    Preconditions.checkState(base != null, "Invalid base metadata: null");
+
+    UpdateTableRequest request =
+        UpdateTableRequest.create(null, ImmutableList.of(), metadata.changes());

Review Comment:
   Can't we use `AssertTableUUID` instead of omitting the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1860,6 +2320,26 @@ components:
               items:
                 type: string
 
+    AddViewVersionUpdate:
+      allOf:
+        - $ref: '#/components/schemas/BaseUpdate'
+        - type: object
+          required:
+            - view-version
+          properties:
+            view-version:
+              $ref: '#/components/schemas/ViewVersion'
+
+    SetCurrentViewVersionUpdate:
+      allOf:
+        - $ref: '#/components/schemas/BaseUpdate'
+        - type: object
+          required:
+            - view-version-id

Review Comment:
   This also needs to support -1 to signal that the version should be the last added version 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -971,4 +996,261 @@ public void commitTransaction(SessionContext context, List<TableCommit> commits)
         headers(context),
         ErrorHandlers.tableCommitHandler());
   }
+
+  @Override
+  public List<TableIdentifier> listViews(SessionContext context, Namespace namespace) {
+    checkNamespaceIsValid(namespace);
+
+    ListTablesResponse response =
+        client.get(
+            paths.views(namespace),
+            ListTablesResponse.class,
+            headers(context),
+            ErrorHandlers.namespaceErrorHandler());
+    return response.identifiers();
+  }
+
+  @Override
+  public View loadView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    LoadViewResponse response =
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    ViewMetadata metadata = response.metadata();
+
+    RESTViewOperations ops =
+        new RESTViewOperations(
+            client,
+            paths.view(identifier),
+            session::headers,
+            tableFileIO(context, response.config()),
+            metadata);
+
+    trackFileIO(ops);
+
+    return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+  }
+
+  @Override
+  public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) {
+    return new RESTViewBuilder(context, identifier);
+  }
+
+  @Override
+  public boolean dropView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    try {
+      client.delete(
+          paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler());
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) {
+    checkViewIdentifierIsValid(from);
+    checkViewIdentifierIsValid(to);
+
+    RenameTableRequest request =
+        RenameTableRequest.builder().withSource(from).withDestination(to).build();
+
+    client.post(
+        paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler());
+  }
+
+  private class RESTViewBuilder implements ViewBuilder {
+    private final SessionContext context;
+    private final TableIdentifier identifier;
+    private final Map<String, String> properties = Maps.newHashMap();
+    private final List<ViewRepresentation> representations = Lists.newArrayList();
+    private Namespace defaultNamespace = null;
+    private String defaultCatalog = null;
+    private Schema schema = null;
+    private String location = null;
+
+    private RESTViewBuilder(SessionContext context, TableIdentifier identifier) {
+      checkViewIdentifierIsValid(identifier);
+      this.identifier = identifier;
+      this.context = context;
+    }
+
+    @Override
+    public ViewBuilder withSchema(Schema newSchema) {
+      this.schema = newSchema;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withQuery(String dialect, String sql) {
+      representations.add(
+          ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build());
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultCatalog(String catalog) {
+      this.defaultCatalog = catalog;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultNamespace(Namespace namespace) {
+      this.defaultNamespace = namespace;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperties(Map<String, String> newProperties) {
+      this.properties.putAll(newProperties);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperty(String key, String value) {
+      this.properties.put(key, value);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withLocation(String newLocation) {
+      this.location = newLocation;
+      return this;
+    }
+
+    @Override
+    public View create() {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot create view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot create view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot create view without specifying a default namespace");
+
+      ViewVersion viewVersion =
+          ImmutableViewVersion.builder()
+              .versionId(1)
+              .schemaId(schema.schemaId())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .defaultCatalog(defaultCatalog)
+              .timestampMillis(System.currentTimeMillis())
+              .putSummary("operation", "create")

Review Comment:
   No longer needed now that the other PR was merged.



##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -971,4 +996,261 @@ public void commitTransaction(SessionContext context, List<TableCommit> commits)
         headers(context),
         ErrorHandlers.tableCommitHandler());
   }
+
+  @Override
+  public List<TableIdentifier> listViews(SessionContext context, Namespace namespace) {
+    checkNamespaceIsValid(namespace);
+
+    ListTablesResponse response =
+        client.get(
+            paths.views(namespace),
+            ListTablesResponse.class,
+            headers(context),
+            ErrorHandlers.namespaceErrorHandler());
+    return response.identifiers();
+  }
+
+  @Override
+  public View loadView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    LoadViewResponse response =
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    ViewMetadata metadata = response.metadata();
+
+    RESTViewOperations ops =
+        new RESTViewOperations(
+            client,
+            paths.view(identifier),
+            session::headers,
+            tableFileIO(context, response.config()),
+            metadata);
+
+    trackFileIO(ops);
+
+    return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+  }
+
+  @Override
+  public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) {
+    return new RESTViewBuilder(context, identifier);
+  }
+
+  @Override
+  public boolean dropView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    try {
+      client.delete(
+          paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler());
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) {
+    checkViewIdentifierIsValid(from);
+    checkViewIdentifierIsValid(to);
+
+    RenameTableRequest request =
+        RenameTableRequest.builder().withSource(from).withDestination(to).build();
+
+    client.post(
+        paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler());
+  }
+
+  private class RESTViewBuilder implements ViewBuilder {
+    private final SessionContext context;
+    private final TableIdentifier identifier;
+    private final Map<String, String> properties = Maps.newHashMap();
+    private final List<ViewRepresentation> representations = Lists.newArrayList();
+    private Namespace defaultNamespace = null;
+    private String defaultCatalog = null;
+    private Schema schema = null;
+    private String location = null;
+
+    private RESTViewBuilder(SessionContext context, TableIdentifier identifier) {
+      checkViewIdentifierIsValid(identifier);
+      this.identifier = identifier;
+      this.context = context;
+    }
+
+    @Override
+    public ViewBuilder withSchema(Schema newSchema) {
+      this.schema = newSchema;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withQuery(String dialect, String sql) {
+      representations.add(
+          ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build());
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultCatalog(String catalog) {
+      this.defaultCatalog = catalog;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultNamespace(Namespace namespace) {
+      this.defaultNamespace = namespace;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperties(Map<String, String> newProperties) {
+      this.properties.putAll(newProperties);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperty(String key, String value) {
+      this.properties.put(key, value);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withLocation(String newLocation) {
+      this.location = newLocation;
+      return this;
+    }
+
+    @Override
+    public View create() {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot create view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot create view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot create view without specifying a default namespace");
+
+      ViewVersion viewVersion =
+          ImmutableViewVersion.builder()
+              .versionId(1)
+              .schemaId(schema.schemaId())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .defaultCatalog(defaultCatalog)
+              .timestampMillis(System.currentTimeMillis())
+              .putSummary("operation", "create")
+              .build();
+
+      CreateViewRequest request =
+          ImmutableCreateViewRequest.builder()
+              .name(identifier.name())
+              .location(location)
+              .schema(schema)
+              .viewVersion(viewVersion)
+              .properties(properties)
+              .build();
+
+      LoadViewResponse response =
+          client.post(
+              paths.views(identifier.namespace()),
+              request,
+              LoadViewResponse.class,
+              headers(context),
+              ErrorHandlers.viewErrorHandler());
+
+      return viewFromResponse(response);
+    }
+
+    @Override
+    public View createOrReplace() {
+      try {
+        return replace(loadView());
+      } catch (NoSuchViewException e) {
+        return create();
+      }
+    }
+
+    @Override
+    public View replace() {
+      return replace(loadView());
+    }
+
+    private LoadViewResponse loadView() {
+      return client.get(
+          paths.view(identifier),
+          LoadViewResponse.class,
+          headers(context),
+          ErrorHandlers.viewErrorHandler());
+    }
+
+    private View replace(LoadViewResponse response) {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot replace view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot replace view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot replace view without specifying a default namespace");
+
+      ViewMetadata metadata = response.metadata();
+
+      int maxVersionId =
+          metadata.versions().stream()
+              .map(ViewVersion::versionId)
+              .max(Integer::compareTo)
+              .orElseGet(metadata::currentVersionId);
+
+      ViewVersion viewVersion =
+          ImmutableViewVersion.builder()
+              .versionId(maxVersionId + 1)
+              .schemaId(schema.schemaId())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .defaultCatalog(defaultCatalog)
+              .timestampMillis(System.currentTimeMillis())
+              .putSummary("operation", "replace")

Review Comment:
   No longer needed 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +386,128 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  private static BaseView baseView(View view) {

Review Comment:
   A better name would be `asBaseView` so it's clear that this is just a cast.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1056,6 +1056,357 @@ paths:
                 }
               }
 
+  /v1/{prefix}/namespaces/{namespace}/views:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    get:
+      tags:
+        - Catalog API
+      summary: List all view identifiers underneath a given namespace
+      description: Return all view identifiers under this namespace
+      operationId: listViews
+      responses:
+        200:
+          $ref: '#/components/responses/ListTablesResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Create a view in the given namespace
+      description:
+        Create a view in the given namespace.
+      operationId: createView
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CreateViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        409:
+          description: Conflict - The view already exists
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceAlreadyExists:
+                  $ref: '#/components/examples/ViewAlreadyExistsError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/namespaces/{namespace}/views/{view}:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+      - $ref: '#/components/parameters/view'
+
+    get:
+      tags:
+        - Catalog API
+      summary: Load a view from the catalog
+      operationId: loadView
+      description:
+        Load a view from the catalog.
+
+
+        The response contains both configuration and view metadata. The configuration, if non-empty is used
+        as additional configuration for the view that overrides catalog configuration. For example, this
+        configuration may change the FileIO implementation to be used for the view.
+
+
+        The response also contains the view's full metadata, matching the view metadata JSON file.
+
+
+        The catalog configuration may contain credentials that should be used for subsequent requests for the
+        view. The configuration key "token" is used to pass an access token to be used as a bearer token
+        for view requests. Otherwise, a token may be passed using a RFC 8693 token type as a configuration
+        key. For example, "urn:ietf:params:oauth:token-type:jwt=<JWT-token>".
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToLoadDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Replace a view
+      operationId: replaceView
+      description:
+        Commit updates to a view.
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToUpdateDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        409:
+          description:
+            Conflict - CommitFailedException. The client may retry.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        500:
+          description:
+            An unknown server-side problem occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Internal Server Error",
+                  "type": "CommitStateUnknownException",
+                  "code": 500
+                }
+              }
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        502:
+          description:
+            A gateway or proxy received an invalid response from the upstream server; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Invalid response from the upstream server",
+                  "type": "CommitStateUnknownException",
+                  "code": 502
+                }
+              }
+        504:
+          description:
+            A server-side gateway timeout occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Gateway timed out during commit",
+                  "type": "CommitStateUnknownException",
+                  "code": 504
+                }
+              }
+        5XX:
+          description:
+            A server-side problem that might not be addressable on the client.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Bad Gateway",
+                  "type": "InternalServerError",
+                  "code": 502
+                }
+              }
+
+    delete:
+      tags:
+        - Catalog API
+      summary: Drop a view from the catalog
+      operationId: dropView
+      description: Remove a view from the catalog
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to drop does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToDeleteDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    head:
+      tags:
+        - Catalog API
+      summary: Check if a view exists
+      operationId: viewExists
+      description:
+        Check if a view exists within a given namespace. This request does not return a response body.
+      responses:
+        200:

Review Comment:
   Should this be 204? Or at least allow 204?



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -971,4 +996,261 @@ public void commitTransaction(SessionContext context, List<TableCommit> commits)
         headers(context),
         ErrorHandlers.tableCommitHandler());
   }
+
+  @Override
+  public List<TableIdentifier> listViews(SessionContext context, Namespace namespace) {
+    checkNamespaceIsValid(namespace);
+
+    ListTablesResponse response =
+        client.get(
+            paths.views(namespace),
+            ListTablesResponse.class,
+            headers(context),
+            ErrorHandlers.namespaceErrorHandler());
+    return response.identifiers();
+  }
+
+  @Override
+  public View loadView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    LoadViewResponse response =
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    ViewMetadata metadata = response.metadata();
+
+    RESTViewOperations ops =
+        new RESTViewOperations(
+            client,
+            paths.view(identifier),
+            session::headers,
+            tableFileIO(context, response.config()),
+            metadata);
+
+    trackFileIO(ops);
+
+    return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+  }
+
+  @Override
+  public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) {
+    return new RESTViewBuilder(context, identifier);
+  }
+
+  @Override
+  public boolean dropView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    try {
+      client.delete(
+          paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler());
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) {
+    checkViewIdentifierIsValid(from);
+    checkViewIdentifierIsValid(to);
+
+    RenameTableRequest request =
+        RenameTableRequest.builder().withSource(from).withDestination(to).build();
+
+    client.post(
+        paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler());
+  }
+
+  private class RESTViewBuilder implements ViewBuilder {
+    private final SessionContext context;
+    private final TableIdentifier identifier;
+    private final Map<String, String> properties = Maps.newHashMap();
+    private final List<ViewRepresentation> representations = Lists.newArrayList();
+    private Namespace defaultNamespace = null;
+    private String defaultCatalog = null;
+    private Schema schema = null;
+    private String location = null;
+
+    private RESTViewBuilder(SessionContext context, TableIdentifier identifier) {
+      checkViewIdentifierIsValid(identifier);
+      this.identifier = identifier;
+      this.context = context;
+    }
+
+    @Override
+    public ViewBuilder withSchema(Schema newSchema) {
+      this.schema = newSchema;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withQuery(String dialect, String sql) {
+      representations.add(
+          ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build());
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultCatalog(String catalog) {
+      this.defaultCatalog = catalog;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultNamespace(Namespace namespace) {
+      this.defaultNamespace = namespace;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperties(Map<String, String> newProperties) {
+      this.properties.putAll(newProperties);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperty(String key, String value) {
+      this.properties.put(key, value);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withLocation(String newLocation) {
+      this.location = newLocation;
+      return this;
+    }
+
+    @Override
+    public View create() {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot create view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot create view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot create view without specifying a default namespace");
+
+      ViewVersion viewVersion =
+          ImmutableViewVersion.builder()
+              .versionId(1)
+              .schemaId(schema.schemaId())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .defaultCatalog(defaultCatalog)
+              .timestampMillis(System.currentTimeMillis())
+              .putSummary("operation", "create")
+              .build();
+
+      CreateViewRequest request =
+          ImmutableCreateViewRequest.builder()
+              .name(identifier.name())
+              .location(location)
+              .schema(schema)
+              .viewVersion(viewVersion)
+              .properties(properties)
+              .build();
+
+      LoadViewResponse response =
+          client.post(
+              paths.views(identifier.namespace()),
+              request,
+              LoadViewResponse.class,
+              headers(context),
+              ErrorHandlers.viewErrorHandler());
+
+      return viewFromResponse(response);
+    }
+
+    @Override
+    public View createOrReplace() {
+      try {
+        return replace(loadView());
+      } catch (NoSuchViewException e) {
+        return create();
+      }
+    }
+
+    @Override
+    public View replace() {
+      return replace(loadView());
+    }
+
+    private LoadViewResponse loadView() {
+      return client.get(
+          paths.view(identifier),
+          LoadViewResponse.class,
+          headers(context),
+          ErrorHandlers.viewErrorHandler());
+    }
+
+    private View replace(LoadViewResponse response) {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot replace view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot replace view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot replace view without specifying a default namespace");
+
+      ViewMetadata metadata = response.metadata();
+
+      int maxVersionId =
+          metadata.versions().stream()
+              .map(ViewVersion::versionId)
+              .max(Integer::compareTo)
+              .orElseGet(metadata::currentVersionId);
+
+      ViewVersion viewVersion =
+          ImmutableViewVersion.builder()
+              .versionId(maxVersionId + 1)
+              .schemaId(schema.schemaId())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .defaultCatalog(defaultCatalog)
+              .timestampMillis(System.currentTimeMillis())
+              .putSummary("operation", "replace")
+              .build();
+
+      ViewMetadata.Builder builder =
+          ViewMetadata.buildFrom(metadata)
+              .setProperties(properties)
+              .setCurrentVersion(viewVersion, schema);
+
+      if (null != location) {
+        builder.setLocation(location);
+      }
+
+      ViewMetadata replacement = builder.build();
+
+      UpdateTableRequest request =
+          UpdateTableRequest.create(identifier, ImmutableList.of(), replacement.changes());

Review Comment:
   I don't think this should create its own request and call the route directly. This duplicates logic that is in `RESTViewOperations`. Instead, this should create `RESTViewOperations` initially and then use it to commit the replacement metadata.



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));

Review Comment:
   This is another problem. If there are representations that are not supported, this should reject them, not silently ignore them.



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -84,6 +86,11 @@ public TableBuilder buildTable(TableIdentifier ident, Schema schema) {
       return BaseSessionCatalog.this.buildTable(context, ident, schema);
     }
 
+    @Override
+    public void initialize(String catalogName, Map<String, String> props) {
+      BaseSessionCatalog.this.initialize(catalogName, props);

Review Comment:
   This would re-initialize the wrapped `SessionCatalog` so I don't think that this is correct. Instead, this should throw an exception that complains about attempting to initialize the catalog when the underlying session catalog has been initialized.
   
   When used through `RESTCatalog`, this is never called.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();

Review Comment:
   agreed, I was planning to update `CreateViewRequest` to be change-based rather than sending the full metadata. That's also why I marked the PR as WIP to indicate there's some additional work required



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -283,20 +292,25 @@ private int addVersionInternal(ViewVersion version) {
         }
       }
 
-      ViewVersion newVersion;
       if (newVersionId != version.versionId()) {
-        newVersion = ImmutableViewVersion.builder().from(version).versionId(newVersionId).build();
+        version = ImmutableViewVersion.builder().from(version).versionId(newVersionId).build();
+      }

Review Comment:
   Nit: Now that the version may be updated more than once, it probably makes more sense to relocate this to just after the new version ID is assigned.



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1752,6 +2223,26 @@ components:
               items:
                 type: string
 
+    AddViewVersionUpdate:
+      allOf:
+        - $ref: '#/components/schemas/BaseUpdate'
+        - type: object
+          required:
+            - view-version
+          properties:
+            view-version:
+              $ref: '#/components/schemas/ViewVersion'
+
+    SetCurrentViewVersionUpdate:
+      allOf:
+        - $ref: '#/components/schemas/BaseUpdate'
+        - type: object
+          required:
+            - view-version-id
+          properties:
+            view-version-id:
+              type: integer
+
     TableUpdate:

Review Comment:
   makes sense, I've updated this in the latest commit



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));
+    View view = viewBuilder.create();
+
+    return viewResponse(view);
+  }
+
+  private static LoadViewResponse viewResponse(View view) {
+    ViewMetadata metadata = baseView(view).operations().current();
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    View view = catalog.loadView(viewIdentifier);
+    return viewResponse(view);
+  }
+
+  public static LoadViewResponse updateView(
+      ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) {
+    View view = catalog.loadView(ident);
+    ViewMetadata metadata = commit(baseView(view).operations(), request);
+
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static void renameView(ViewCatalog catalog, RenameTableRequest request) {
+    catalog.renameView(request.source(), request.destination());
+  }
+
+  public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    boolean dropped = catalog.dropView(viewIdentifier);
+    if (!dropped) {
+      throw new NoSuchViewException("View does not exist: %s", viewIdentifier);
+    }
+  }
+
+  static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
+    AtomicBoolean isRetry = new AtomicBoolean(false);
+    try {
+      Tasks.foreach(ops)
+          .retry(COMMIT_NUM_RETRIES_DEFAULT)
+          .exponentialBackoff(
+              COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
+              2.0 /* exponential */)
+          .onlyRetryOn(CommitFailedException.class)
+          .run(
+              taskOps -> {
+                ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current();
+                isRetry.set(true);
+
+                // apply changes
+                ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base);
+                request.updates().forEach(update -> update.applyTo(metadataBuilder));

Review Comment:
   > 1.) For creating a view, there probably needs to be a requirement similar to table's AssertTableDoesNotExist but for views, AssertViewDoesNotExist. Similar principle for ReplaceView and verifying the UUID as expected.
   
   For tables I believe `UpdateRequirement.AssertTableDoesNotExist` is mainly used for transactional cases, whereas for views we don't have transaction support. I think what would make sense is to have an assertion on the UUID of the view.
   
   > 2.) I don't see a metadata update for AddViewRepresentation but I'd imagine that should validate that the SQL dialect does not already exist for the specified version. I forgot where we landed on that discussion, We do want to prevent that right?
   
   We do have `MetadataUpdate.AddViewVersion` for this and we do validate now that there's no duplicate dialect (which was fixed as part of #7880), so we should be good here.
   
   > 3.) A requirement for the schema ID to make sure that is unchanged from the base metadata?
   
   Typically this would be done via `UpdateRequirement.AssertCurrentSchemaID` when there's a `MetadataUpdate.SetCurrentSchema` update, but for Views we don't send a `MetadataUpdate.SetCurrentSchema` 



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1056,6 +1056,357 @@ paths:
                 }
               }
 
+  /v1/{prefix}/namespaces/{namespace}/views:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    get:
+      tags:
+        - Catalog API
+      summary: List all view identifiers underneath a given namespace
+      description: Return all view identifiers under this namespace
+      operationId: listViews
+      responses:
+        200:
+          $ref: '#/components/responses/ListTablesResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Create a view in the given namespace
+      description:
+        Create a view in the given namespace.
+      operationId: createView
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CreateViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        409:
+          description: Conflict - The view already exists
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceAlreadyExists:
+                  $ref: '#/components/examples/ViewAlreadyExistsError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/namespaces/{namespace}/views/{view}:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+      - $ref: '#/components/parameters/view'
+
+    get:
+      tags:
+        - Catalog API
+      summary: Load a view from the catalog
+      operationId: loadView
+      description:
+        Load a view from the catalog.
+
+
+        The response contains both configuration and view metadata. The configuration, if non-empty is used
+        as additional configuration for the view that overrides catalog configuration. For example, this
+        configuration may change the FileIO implementation to be used for the view.

Review Comment:
   I can't think of a good example, so I've left the example out until we have something better to mention 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +386,128 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  private static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(request.schema())
+            .withProperties(request.properties())
+            .withDefaultNamespace(request.viewVersion().defaultNamespace())
+            .withDefaultCatalog(request.viewVersion().defaultCatalog())
+            .withLocation(request.location());
+
+    Set<String> unsupportedRepresentations =
+        request.viewVersion().representations().stream()
+            .filter(r -> !(r instanceof SQLViewRepresentation))
+            .map(ViewRepresentation::type)
+            .collect(Collectors.toSet());
+
+    if (!unsupportedRepresentations.isEmpty()) {
+      throw new IllegalStateException(
+          String.format("Found unsupported view representations: %s", unsupportedRepresentations));
+    }
+
+    request.viewVersion().representations().stream()

Review Comment:
   Note that this ignores the schema ID of the view version. I'm okay with that.



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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));
+    View view = viewBuilder.create();
+
+    return viewResponse(view);
+  }
+
+  private static LoadViewResponse viewResponse(View view) {
+    ViewMetadata metadata = baseView(view).operations().current();
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    View view = catalog.loadView(viewIdentifier);
+    return viewResponse(view);
+  }
+
+  public static LoadViewResponse updateView(
+      ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) {
+    View view = catalog.loadView(ident);
+    ViewMetadata metadata = commit(baseView(view).operations(), request);
+
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static void renameView(ViewCatalog catalog, RenameTableRequest request) {
+    catalog.renameView(request.source(), request.destination());
+  }
+
+  public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    boolean dropped = catalog.dropView(viewIdentifier);
+    if (!dropped) {
+      throw new NoSuchViewException("View does not exist: %s", viewIdentifier);
+    }
+  }
+
+  static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
+    AtomicBoolean isRetry = new AtomicBoolean(false);
+    try {
+      Tasks.foreach(ops)
+          .retry(COMMIT_NUM_RETRIES_DEFAULT)
+          .exponentialBackoff(
+              COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
+              2.0 /* exponential */)
+          .onlyRetryOn(CommitFailedException.class)
+          .run(
+              taskOps -> {
+                ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current();
+                isRetry.set(true);
+
+                // apply changes
+                ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base);
+                request.updates().forEach(update -> update.applyTo(metadataBuilder));

Review Comment:
   based on https://github.com/apache/iceberg/pull/8147#discussion_r1274225077 we decided that we don't need view requirements (for now), hence this code wasn't validating any requirements. I guess we can re-visit and see what requirements would make sense to have for views



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +386,128 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  private static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");

Review Comment:
   done



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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/UpdateRequirement.java:
##########
@@ -20,11 +20,17 @@
 
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.view.ViewMetadata;
 
 /** Represents a requirement for a {@link MetadataUpdate} */
 public interface UpdateRequirement {
   void validate(TableMetadata base);
 
+  default void validate(ViewMetadata base) {
+    throw new UnsupportedOperationException(

Review Comment:
   makes sense to use `ValidationException` here, I've done that



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

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] danielcweeks commented on a diff in pull request #7913: Core: Add View support for REST catalog

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1752,6 +2223,26 @@ components:
               items:
                 type: string
 
+    AddViewVersionUpdate:
+      allOf:
+        - $ref: '#/components/schemas/BaseUpdate'
+        - type: object
+          required:
+            - view-version
+          properties:
+            view-version:
+              $ref: '#/components/schemas/ViewVersion'
+
+    SetCurrentViewVersionUpdate:
+      allOf:
+        - $ref: '#/components/schemas/BaseUpdate'
+        - type: object
+          required:
+            - view-version-id
+          properties:
+            view-version-id:
+              type: integer
+
     TableUpdate:

Review Comment:
   We might want to split this out into `ViewUpdate` to keep the operations specific to the endpoints.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -971,4 +996,250 @@ public void commitTransaction(SessionContext context, List<TableCommit> commits)
         headers(context),
         ErrorHandlers.tableCommitHandler());
   }
+
+  @Override
+  public List<TableIdentifier> listViews(SessionContext context, Namespace namespace) {
+    checkNamespaceIsValid(namespace);
+
+    ListTablesResponse response =
+        client.get(
+            paths.views(namespace),
+            ListTablesResponse.class,
+            headers(context),
+            ErrorHandlers.namespaceErrorHandler());
+    return response.identifiers();
+  }
+
+  @Override
+  public View loadView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    LoadViewResponse response =
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    ViewMetadata metadata = response.metadata();
+
+    RESTViewOperations ops =
+        new RESTViewOperations(
+            client,
+            paths.view(identifier),
+            session::headers,
+            tableFileIO(context, response.config()),
+            metadata);
+
+    trackFileIO(ops);
+
+    return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+  }
+
+  @Override
+  public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) {
+    return new RESTViewBuilder(context, identifier);
+  }
+
+  @Override
+  public boolean dropView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    try {
+      client.delete(
+          paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler());
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) {
+    checkViewIdentifierIsValid(from);
+    checkViewIdentifierIsValid(to);
+
+    RenameTableRequest request =
+        RenameTableRequest.builder().withSource(from).withDestination(to).build();
+
+    client.post(
+        paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler());
+  }
+
+  private class RESTViewBuilder implements ViewBuilder {
+    private final SessionContext context;
+    private final TableIdentifier identifier;
+    private final ImmutableViewVersion.Builder viewVersionBuilder = ImmutableViewVersion.builder();

Review Comment:
   yes, this PR hasn't been updated yet to reflect the new changes we did for #7880 



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -2033,6 +2537,68 @@ components:
         metadata-location:
           type: string
 
+    CreateViewRequest:
+      type: object
+      required:
+        - name
+        - schema
+        - view-version
+        - properties
+      properties:
+        name:
+          type: string
+        location:
+          type: string
+        schema:
+          $ref: '#/components/schemas/Schema'
+        view-version:
+          $ref: '#/components/schemas/ViewVersion'

Review Comment:
   This will need to replace the `schema-id` sent with the `ViewVersion` with whatever was assigned to the schema above.



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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));
+    View view = viewBuilder.create();
+
+    return viewResponse(view);
+  }
+
+  private static LoadViewResponse viewResponse(View view) {
+    ViewMetadata metadata = baseView(view).operations().current();
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    View view = catalog.loadView(viewIdentifier);
+    return viewResponse(view);
+  }
+
+  public static LoadViewResponse updateView(
+      ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) {
+    View view = catalog.loadView(ident);
+    ViewMetadata metadata = commit(baseView(view).operations(), request);
+
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static void renameView(ViewCatalog catalog, RenameTableRequest request) {
+    catalog.renameView(request.source(), request.destination());
+  }
+
+  public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    boolean dropped = catalog.dropView(viewIdentifier);
+    if (!dropped) {
+      throw new NoSuchViewException("View does not exist: %s", viewIdentifier);
+    }
+  }
+
+  static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
+    AtomicBoolean isRetry = new AtomicBoolean(false);
+    try {
+      Tasks.foreach(ops)
+          .retry(COMMIT_NUM_RETRIES_DEFAULT)
+          .exponentialBackoff(
+              COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
+              2.0 /* exponential */)
+          .onlyRetryOn(CommitFailedException.class)
+          .run(
+              taskOps -> {
+                ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current();
+                isRetry.set(true);
+
+                // apply changes
+                ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base);
+                request.updates().forEach(update -> update.applyTo(metadataBuilder));

Review Comment:
   I've opened https://github.com/apache/iceberg/pull/8831 to introduce `AssertUUID` (and deprecate `AssertTableUUID`) so that we can use it both for tables and views. Alternatively, we could also introduce `AssertViewUUID` with the same functionality as `AssertTableUUID`



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -283,20 +292,25 @@ private int addVersionInternal(ViewVersion version) {
         }
       }
 
-      ViewVersion newVersion;
       if (newVersionId != version.versionId()) {
-        newVersion = ImmutableViewVersion.builder().from(version).versionId(newVersionId).build();
+        version = ImmutableViewVersion.builder().from(version).versionId(newVersionId).build();
+      }

Review Comment:
   done



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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -1534,4 +1535,90 @@ public void updateViewLocationConflict() {
         .isInstanceOf(NoSuchViewException.class)
         .hasMessageContaining("View does not exist: ns.view");
   }
+
+  @Test
+  public void concurrentReplaceViewVersion() {
+    TableIdentifier identifier = TableIdentifier.of("ns", "view");
+
+    if (requiresNamespaceCreate()) {
+      catalog().createNamespace(identifier.namespace());
+    }
+
+    assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse();
+
+    View view =
+        catalog()
+            .buildView(identifier)
+            .withSchema(SCHEMA)
+            .withDefaultNamespace(identifier.namespace())
+            .withQuery("trino", "select * from ns.tbl")
+            .create();
+
+    assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue();
+
+    ReplaceViewVersion replaceViewVersion =
+        view.replaceVersion()
+            .withQuery("trino", "select count(*) from ns.tbl")
+            .withSchema(SCHEMA)
+            .withDefaultNamespace(identifier.namespace());
+
+    ReplaceViewVersion replaceViewVersionConcurrent =
+        view.replaceVersion()
+            .withQuery("spark", "select count(*) from ns.tbl")
+            .withSchema(OTHER_SCHEMA)
+            .withDefaultNamespace(identifier.namespace());
+
+    // concurrently replace the view version, the last replace wins
+    replaceViewVersionConcurrent.commit();

Review Comment:
   Minor: the names don't make sense to me here. Both of these are "concurrent" operations that we are simulating. The relevant difference is that this one uses `OTHER_SCHEMA`.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -1534,4 +1535,90 @@ public void updateViewLocationConflict() {
         .isInstanceOf(NoSuchViewException.class)
         .hasMessageContaining("View does not exist: ns.view");
   }
+
+  @Test
+  public void concurrentReplaceViewVersion() {
+    TableIdentifier identifier = TableIdentifier.of("ns", "view");
+
+    if (requiresNamespaceCreate()) {
+      catalog().createNamespace(identifier.namespace());
+    }
+
+    assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse();
+
+    View view =
+        catalog()
+            .buildView(identifier)
+            .withSchema(SCHEMA)
+            .withDefaultNamespace(identifier.namespace())
+            .withQuery("trino", "select * from ns.tbl")
+            .create();
+
+    assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue();
+
+    ReplaceViewVersion replaceViewVersion =
+        view.replaceVersion()
+            .withQuery("trino", "select count(*) from ns.tbl")
+            .withSchema(SCHEMA)
+            .withDefaultNamespace(identifier.namespace());
+
+    ReplaceViewVersion replaceViewVersionConcurrent =
+        view.replaceVersion()
+            .withQuery("spark", "select count(*) from ns.tbl")
+            .withSchema(OTHER_SCHEMA)
+            .withDefaultNamespace(identifier.namespace());
+
+    // concurrently replace the view version, the last replace wins
+    replaceViewVersionConcurrent.commit();

Review Comment:
   I've slightly renamed 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())

Review Comment:
   This shouldn't access `schema` directly. It should use the schema of the current version.



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1052,6 +1403,15 @@ components:
         type: string
       example: "sales"
 
+    view:
+      name: view
+      in: path
+      description: A view name
+      required: true
+      schema:
+        type: string
+      example: "sales-view"

Review Comment:
   No need for the suffix `-view`.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -943,11 +968,11 @@ private static Cache<String, AuthSession> newSessionCache(Map<String, String> pr
         .build();
   }
 
-  private Cache<TableOperations, FileIO> newFileIOCloser() {
+  private Cache<Object, FileIO> newFileIOCloser() {

Review Comment:
   Since this doesn't need to create `FileIO` instances for views, I don't think this change is needed. Right?



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1056,6 +1056,357 @@ paths:
                 }
               }
 
+  /v1/{prefix}/namespaces/{namespace}/views:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    get:
+      tags:
+        - Catalog API
+      summary: List all view identifiers underneath a given namespace
+      description: Return all view identifiers under this namespace
+      operationId: listViews
+      responses:
+        200:
+          $ref: '#/components/responses/ListTablesResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Create a view in the given namespace
+      description:
+        Create a view in the given namespace.
+      operationId: createView
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CreateViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        409:
+          description: Conflict - The view already exists
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceAlreadyExists:
+                  $ref: '#/components/examples/ViewAlreadyExistsError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/namespaces/{namespace}/views/{view}:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+      - $ref: '#/components/parameters/view'
+
+    get:
+      tags:
+        - Catalog API
+      summary: Load a view from the catalog
+      operationId: loadView
+      description:
+        Load a view from the catalog.
+
+
+        The response contains both configuration and view metadata. The configuration, if non-empty is used
+        as additional configuration for the view that overrides catalog configuration. For example, this
+        configuration may change the FileIO implementation to be used for the view.

Review Comment:
   We may want a different example for this since views don't need or expose a `FileIO`.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1630,6 +1990,102 @@ components:
         metadata-log:
           $ref: '#/components/schemas/MetadataLog'
 
+    SQLViewRepresentation:
+      type: object
+      required:
+        - type
+        - sql
+        - dialect
+      properties:
+        type:
+          type: string
+        sql:
+          type: string
+        dialect:
+          type: string
+
+    ViewRepresentation:
+      oneOf:
+        - $ref: '#/components/schemas/SQLViewRepresentation'
+
+    ViewHistoryEntry:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+
+    ViewVersion:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+        - schema-id
+        - summary
+        - representations
+        - default-namespace
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+        schema-id:
+          type: integer

Review Comment:
   @nastra, we need to replace the schema ID and pass -1 here if the schema was just added. The server side may reassign the schema ID so it needs to be replaced with a placeholder.
   
   We will also need a test for this behavior where the view is updated concurrently. Server-side retry should translate the schema IDs correctly.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -2033,6 +2537,68 @@ components:
         metadata-location:
           type: string
 
+    CreateViewRequest:
+      type: object
+      required:
+        - name
+        - schema
+        - view-version
+        - properties
+      properties:
+        name:
+          type: string
+        location:
+          type: string
+        schema:
+          $ref: '#/components/schemas/Schema'
+        view-version:
+          $ref: '#/components/schemas/ViewVersion'
+        properties:
+          type: object
+          additionalProperties:
+            type: string
+
+    LoadViewResult:
+      description: |
+        Result used when a view is successfully loaded.
+
+
+        The view metadata JSON is returned in the `metadata` field. The corresponding file location of view metadata is returned in the `metadata-location` field.

Review Comment:
   Looks like this line is too long. Can you check it against the rest of the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -2033,6 +2537,68 @@ components:
         metadata-location:
           type: string
 
+    CreateViewRequest:
+      type: object
+      required:
+        - name
+        - schema
+        - view-version
+        - properties
+      properties:
+        name:
+          type: string
+        location:
+          type: string
+        schema:
+          $ref: '#/components/schemas/Schema'
+        view-version:
+          $ref: '#/components/schemas/ViewVersion'

Review Comment:
   I've added a description to make this clearer



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/UpdateRequirement.java:
##########
@@ -20,11 +20,17 @@
 
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.view.ViewMetadata;
 
 /** Represents a requirement for a {@link MetadataUpdate} */
 public interface UpdateRequirement {
   void validate(TableMetadata base);
 
+  default void validate(ViewMetadata base) {
+    throw new UnsupportedOperationException(

Review Comment:
   my bad, I forgot to push those changes



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


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

Review Comment:
   I've changed things to use a diff-based approach (similar to how we have it for TableMetadata)



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1630,6 +1990,102 @@ components:
         metadata-log:
           $ref: '#/components/schemas/MetadataLog'
 
+    SQLViewRepresentation:
+      type: object
+      required:
+        - type
+        - sql
+        - dialect
+      properties:
+        type:
+          type: string
+        sql:
+          type: string
+        dialect:
+          type: string
+
+    ViewRepresentation:
+      oneOf:
+        - $ref: '#/components/schemas/SQLViewRepresentation'
+
+    ViewHistoryEntry:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+
+    ViewVersion:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+        - schema-id
+        - summary
+        - representations
+        - default-namespace

Review Comment:
   What about default 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -1444,9 +1522,14 @@ public void updateViewLocationConflict() {
     catalog().dropView(identifier);
     assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse();
 
+    String expectedMessage =
+        catalog() instanceof RESTCatalog ? "View does not exist: ns.view" : "Cannot commit";
+    Class<?> expectedException =
+        catalog() instanceof RESTCatalog ? NoSuchViewException.class : CommitFailedException.class;

Review Comment:
   can you please move the changes in viewCatalogTests to the separate PR and merge it first? 
   I am working on View support for Nessie catalog and these tests are failing for me too. 
   
   Also I think we cannot have a catalog specific check in this class. It will become too clumsy once we extend this test for all the catalogs. Can you please simplify the assert to fail on generic reason?



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest;
+import org.apache.iceberg.rest.responses.LoadViewResponse;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.iceberg.view.ViewOperations;
+
+class RESTViewOperations implements ViewOperations {
+
+  private final RESTClient client;
+  private final String path;
+  private final Supplier<Map<String, String>> headers;
+  private final FileIO io;
+  private ViewMetadata current;
+
+  RESTViewOperations(
+      RESTClient client,
+      String path,
+      Supplier<Map<String, String>> headers,
+      FileIO io,
+      ViewMetadata current) {
+    Preconditions.checkArgument(null != current, "Invalid view metadata: null");
+    this.client = client;
+    this.path = path;
+    this.headers = headers;
+    this.io = io;
+    this.current = current;
+  }
+
+  @Override
+  public ViewMetadata current() {
+    return current;
+  }
+
+  @Override
+  public ViewMetadata refresh() {
+    return updateCurrentMetadata(
+        client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler()));
+  }
+
+  @Override
+  public void commit(ViewMetadata base, ViewMetadata metadata) {
+    // this is only used for replacing view metadata
+    Preconditions.checkState(base != null, "Invalid base metadata: null");
+
+    UpdateTableRequest request =
+        UpdateTableRequest.create(null, ImmutableList.of(), metadata.changes());
+
+    LoadViewResponse response =
+        client.post(
+            path, request, LoadViewResponse.class, headers, ErrorHandlers.viewCommitHandler());
+
+    updateCurrentMetadata(response);
+  }
+
+  public FileIO io() {

Review Comment:
   No need for this or the `io` field.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -30,8 +30,10 @@
 import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.view.View;
+import org.apache.iceberg.view.ViewBuilder;
 
-public abstract class BaseSessionCatalog implements SessionCatalog {
+public abstract class BaseSessionCatalog implements SessionCatalog, ViewSessionCatalog {

Review Comment:
   @nastra any update on why RevAPI didn't catch 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest;
+import org.apache.iceberg.rest.responses.LoadViewResponse;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.iceberg.view.ViewOperations;
+
+class RESTViewOperations implements ViewOperations {
+
+  private final RESTClient client;
+  private final String path;
+  private final Supplier<Map<String, String>> headers;
+  private final FileIO io;
+  private ViewMetadata current;
+
+  RESTViewOperations(
+      RESTClient client,
+      String path,
+      Supplier<Map<String, String>> headers,
+      FileIO io,
+      ViewMetadata current) {
+    Preconditions.checkArgument(null != current, "Invalid view metadata: null");
+    this.client = client;
+    this.path = path;
+    this.headers = headers;
+    this.io = io;
+    this.current = current;
+  }
+
+  @Override
+  public ViewMetadata current() {
+    return current;
+  }
+
+  @Override
+  public ViewMetadata refresh() {
+    return updateCurrentMetadata(
+        client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler()));
+  }
+
+  @Override
+  public void commit(ViewMetadata base, ViewMetadata metadata) {
+    // this is only used for replacing view metadata
+    Preconditions.checkState(base != null, "Invalid base metadata: null");
+
+    UpdateTableRequest request =
+        UpdateTableRequest.create(null, ImmutableList.of(), metadata.changes());
+
+    LoadViewResponse response =
+        client.post(
+            path, request, LoadViewResponse.class, headers, ErrorHandlers.viewCommitHandler());
+
+    updateCurrentMetadata(response);
+  }
+
+  public FileIO io() {

Review Comment:
   ah you're right, this was from an earlier version of the code where it was used. I've removed `FileIO` now from 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/UpdateRequirement.java:
##########
@@ -20,11 +20,17 @@
 
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.view.ViewMetadata;
 
 /** Represents a requirement for a {@link MetadataUpdate} */
 public interface UpdateRequirement {
   void validate(TableMetadata base);
 
+  default void validate(ViewMetadata base) {
+    throw new UnsupportedOperationException(

Review Comment:
   Should we use a `ValidationException` instead since using an assertion on a view in this case is invalid? It isn't that it is unsupported. It's that the validation doesn't apply.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -1534,4 +1535,90 @@ public void updateViewLocationConflict() {
         .isInstanceOf(NoSuchViewException.class)
         .hasMessageContaining("View does not exist: ns.view");
   }
+
+  @Test
+  public void concurrentReplaceViewVersion() {
+    TableIdentifier identifier = TableIdentifier.of("ns", "view");
+
+    if (requiresNamespaceCreate()) {
+      catalog().createNamespace(identifier.namespace());
+    }
+
+    assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse();
+
+    View view =
+        catalog()
+            .buildView(identifier)
+            .withSchema(SCHEMA)

Review Comment:
   It's odd that the schemas don't match the queries.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1056,6 +1056,357 @@ paths:
                 }
               }
 
+  /v1/{prefix}/namespaces/{namespace}/views:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    get:
+      tags:
+        - Catalog API
+      summary: List all view identifiers underneath a given namespace
+      description: Return all view identifiers under this namespace
+      operationId: listViews
+      responses:
+        200:
+          $ref: '#/components/responses/ListTablesResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Create a view in the given namespace
+      description:
+        Create a view in the given namespace.
+      operationId: createView
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CreateViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        409:
+          description: Conflict - The view already exists
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceAlreadyExists:
+                  $ref: '#/components/examples/ViewAlreadyExistsError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/namespaces/{namespace}/views/{view}:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+      - $ref: '#/components/parameters/view'
+
+    get:
+      tags:
+        - Catalog API
+      summary: Load a view from the catalog
+      operationId: loadView
+      description:
+        Load a view from the catalog.
+
+
+        The response contains both configuration and view metadata. The configuration, if non-empty is used
+        as additional configuration for the view that overrides catalog configuration. For example, this
+        configuration may change the FileIO implementation to be used for the view.
+
+
+        The response also contains the view's full metadata, matching the view metadata JSON file.
+
+
+        The catalog configuration may contain credentials that should be used for subsequent requests for the
+        view. The configuration key "token" is used to pass an access token to be used as a bearer token
+        for view requests. Otherwise, a token may be passed using a RFC 8693 token type as a configuration
+        key. For example, "urn:ietf:params:oauth:token-type:jwt=<JWT-token>".
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToLoadDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Replace a view
+      operationId: replaceView
+      description:
+        Commit updates to a view.
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToUpdateDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        409:
+          description:
+            Conflict - CommitFailedException. The client may retry.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        500:
+          description:
+            An unknown server-side problem occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Internal Server Error",
+                  "type": "CommitStateUnknownException",
+                  "code": 500
+                }
+              }
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        502:
+          description:
+            A gateway or proxy received an invalid response from the upstream server; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Invalid response from the upstream server",
+                  "type": "CommitStateUnknownException",
+                  "code": 502
+                }
+              }
+        504:
+          description:
+            A server-side gateway timeout occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Gateway timed out during commit",
+                  "type": "CommitStateUnknownException",
+                  "code": 504
+                }
+              }
+        5XX:
+          description:
+            A server-side problem that might not be addressable on the client.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Bad Gateway",
+                  "type": "InternalServerError",
+                  "code": 502
+                }
+              }
+
+    delete:
+      tags:
+        - Catalog API
+      summary: Drop a view from the catalog
+      operationId: dropView
+      description: Remove a view from the catalog
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to drop does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToDeleteDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    head:
+      tags:
+        - Catalog API
+      summary: Check if a view exists
+      operationId: viewExists
+      description:
+        Check if a view exists within a given namespace. This request does not return a response body.
+      responses:
+        200:

Review Comment:
   yes this should be actually a 204, similar to checking namespace/table existence via a HEAD 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -943,11 +968,11 @@ private static Cache<String, AuthSession> newSessionCache(Map<String, String> pr
         .build();
   }
 
-  private Cache<TableOperations, FileIO> newFileIOCloser() {
+  private Cache<Object, FileIO> newFileIOCloser() {

Review Comment:
   agreed, we can revert these changes and we don't need to track FileIO here for views



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -568,4 +649,9 @@ private static TableIdentifier identFromPathVars(Map<String, String> pathVars) {
     return TableIdentifier.of(
         namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("table")));
   }
+
+  private static TableIdentifier viewIdentFromPathVars(Map<String, String> pathVars) {

Review Comment:
   yep, good idea



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();

Review Comment:
   agreed, I was planning to update `CreateViewRequest` to be change-based rather than sending the full metadata. That's also why I marked the PR as WIP



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -971,4 +996,250 @@ public void commitTransaction(SessionContext context, List<TableCommit> commits)
         headers(context),
         ErrorHandlers.tableCommitHandler());
   }
+
+  @Override
+  public List<TableIdentifier> listViews(SessionContext context, Namespace namespace) {
+    checkNamespaceIsValid(namespace);
+
+    ListTablesResponse response =
+        client.get(
+            paths.views(namespace),
+            ListTablesResponse.class,
+            headers(context),
+            ErrorHandlers.namespaceErrorHandler());
+    return response.identifiers();
+  }
+
+  @Override
+  public View loadView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    LoadViewResponse response =
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    ViewMetadata metadata = response.metadata();
+
+    RESTViewOperations ops =
+        new RESTViewOperations(
+            client,
+            paths.view(identifier),
+            session::headers,
+            tableFileIO(context, response.config()),
+            metadata);
+
+    trackFileIO(ops);
+
+    return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+  }
+
+  @Override
+  public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) {
+    return new RESTViewBuilder(context, identifier);
+  }
+
+  @Override
+  public boolean dropView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    try {
+      client.delete(
+          paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler());
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) {
+    checkViewIdentifierIsValid(from);
+    checkViewIdentifierIsValid(to);
+
+    RenameTableRequest request =
+        RenameTableRequest.builder().withSource(from).withDestination(to).build();
+
+    client.post(
+        paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler());
+  }
+
+  private class RESTViewBuilder implements ViewBuilder {
+    private final SessionContext context;
+    private final TableIdentifier identifier;
+    private final ImmutableViewVersion.Builder viewVersionBuilder = ImmutableViewVersion.builder();
+    private final Map<String, String> properties = Maps.newHashMap();
+    private final List<ViewRepresentation> representations = Lists.newArrayList();
+    private Namespace defaultNamespace = null;
+    private Schema schema = null;
+
+    private RESTViewBuilder(SessionContext context, TableIdentifier identifier) {
+      checkViewIdentifierIsValid(identifier);
+      this.identifier = identifier;
+      this.context = context;
+    }
+
+    @Override
+    public ViewBuilder withSchema(Schema newSchema) {
+      this.schema = newSchema;
+      viewVersionBuilder.schemaId(newSchema.schemaId());
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withQuery(String dialect, String sql) {
+      representations.add(
+          ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build());
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultCatalog(String defaultCatalog) {
+      viewVersionBuilder.defaultCatalog(defaultCatalog);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultNamespace(Namespace namespace) {
+      this.defaultNamespace = namespace;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperties(Map<String, String> newProperties) {
+      this.properties.putAll(newProperties);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperty(String key, String value) {
+      this.properties.put(key, value);
+      return this;
+    }
+
+    @Override
+    public View create() {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot create view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot create view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot create view without specifying a default namespace");
+
+      ViewVersion viewVersion =
+          viewVersionBuilder
+              .versionId(1)
+              .timestampMillis(System.currentTimeMillis())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .putSummary("operation", "create")
+              .build();
+
+      ViewMetadata viewMetadata =
+          ViewMetadata.builder()
+              .setProperties(properties)
+              .setLocation("dummy")
+              .setCurrentVersion(viewVersion, schema)
+              .build();
+
+      CreateViewRequest request =
+          ImmutableCreateViewRequest.builder()
+              .metadata(viewMetadata)
+              .name(identifier.name())
+              .build();
+
+      LoadViewResponse response =
+          client.post(
+              paths.views(identifier.namespace()),
+              request,
+              LoadViewResponse.class,
+              headers(context),
+              ErrorHandlers.viewErrorHandler());
+
+      return viewFromResponse(response);
+    }
+
+    @Override
+    public View replace() {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot replace view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot replace view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot replace view without specifying a default namespace");
+
+      LoadViewResponse response =
+          client.get(
+              paths.view(identifier),
+              LoadViewResponse.class,
+              headers(context),
+              ErrorHandlers.viewErrorHandler());
+
+      ViewMetadata metadata = response.metadata();
+
+      int maxVersionId =
+          metadata.versions().stream()
+              .map(ViewVersion::versionId)
+              .max(Integer::compareTo)
+              .orElseGet(metadata::currentVersionId);
+
+      ViewVersion viewVersion =
+          viewVersionBuilder
+              .versionId(maxVersionId + 1)
+              .timestampMillis(System.currentTimeMillis())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .putSummary("operation", "replace")
+              .build();
+
+      ViewMetadata replacement =
+          ViewMetadata.buildFrom(metadata)
+              .setProperties(properties)
+              .setCurrentVersion(viewVersion, schema)
+              .build();
+
+      UpdateTableRequest request =
+          UpdateTableRequest.create(identifier, ImmutableList.of(), replacement.changes());
+
+      response =
+          client.post(
+              paths.view(identifier),
+              request,
+              LoadViewResponse.class,
+              headers(context),
+              ErrorHandlers.viewErrorHandler());
+
+      return viewFromResponse(response);
+    }
+
+    private BaseView viewFromResponse(LoadViewResponse response) {
+      AuthSession session = tableSession(response.config(), session(context));
+      RESTViewOperations ops =
+          new RESTViewOperations(
+              client,
+              paths.view(identifier),
+              session::headers,
+              tableFileIO(context, response.config()),
+              response.metadata());
+
+      trackFileIO(ops);
+
+      return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+    }
+
+    @Override
+    public View createOrReplace() {
+      try {
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+        return replace();

Review Comment:
   Like the other builder, this should not load the view twice just to determine whether to use replace or create. The replace method should be refactored to accept a LoadViewResponse.



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -568,4 +649,9 @@ private static TableIdentifier identFromPathVars(Map<String, String> pathVars) {
     return TableIdentifier.of(
         namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("table")));
   }
+
+  private static TableIdentifier viewIdentFromPathVars(Map<String, String> pathVars) {

Review Comment:
   Since this is in tests, you could just combine the table and view versions by renaming the variable to `name` right?



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #7913:
URL: https://github.com/apache/iceberg/pull/7913#discussion_r1347539389


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));
+    View view = viewBuilder.create();
+
+    return viewResponse(view);
+  }
+
+  private static LoadViewResponse viewResponse(View view) {
+    ViewMetadata metadata = baseView(view).operations().current();
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    View view = catalog.loadView(viewIdentifier);
+    return viewResponse(view);
+  }
+
+  public static LoadViewResponse updateView(
+      ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) {
+    View view = catalog.loadView(ident);
+    ViewMetadata metadata = commit(baseView(view).operations(), request);
+
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static void renameView(ViewCatalog catalog, RenameTableRequest request) {
+    catalog.renameView(request.source(), request.destination());
+  }
+
+  public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    boolean dropped = catalog.dropView(viewIdentifier);
+    if (!dropped) {
+      throw new NoSuchViewException("View does not exist: %s", viewIdentifier);
+    }
+  }
+
+  static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
+    AtomicBoolean isRetry = new AtomicBoolean(false);
+    try {
+      Tasks.foreach(ops)
+          .retry(COMMIT_NUM_RETRIES_DEFAULT)
+          .exponentialBackoff(
+              COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
+              2.0 /* exponential */)
+          .onlyRetryOn(CommitFailedException.class)
+          .run(
+              taskOps -> {
+                ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current();
+                isRetry.set(true);
+
+                // apply changes
+                ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base);
+                request.updates().forEach(update -> update.applyTo(metadataBuilder));

Review Comment:
   I see we're currently running through the validations on line 486 but if I'm not mistaken right now they're all just no-ops, is that right @nastra ? 
   
   Few requirements that come to mind:
   
   1.) For creating a view, there probably needs to be a requirement similar to table's `AssertTableDoesNotExist` but for views, `AssertViewDoesNotExist`. Similar principle for ReplaceView and verifying the UUID as expected.
   
   
   2.) I don't see a metadata update for AddViewRepresentation but I'd imagine that should validate that the SQL dialect does not already exist for the specified version. I forgot where we landed on that discussion, We do want to prevent that right?
   
   3.) A requirement for the schema ID to make sure that is unchanged from the base metadata? 



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/UpdateRequirement.java:
##########
@@ -20,11 +20,17 @@
 
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.view.ViewMetadata;
 
 /** Represents a requirement for a {@link MetadataUpdate} */
 public interface UpdateRequirement {
   void validate(TableMetadata base);
 
+  default void validate(ViewMetadata base) {
+    throw new UnsupportedOperationException(

Review Comment:
   Should we use a `ValidationException` instead since using an assertion on a view in this case is invalid?



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -2033,6 +2537,68 @@ components:
         metadata-location:
           type: string
 
+    CreateViewRequest:
+      type: object
+      required:
+        - name
+        - schema
+        - view-version
+        - properties
+      properties:
+        name:
+          type: string
+        location:
+          type: string
+        schema:
+          $ref: '#/components/schemas/Schema'
+        view-version:
+          $ref: '#/components/schemas/ViewVersion'
+        properties:
+          type: object
+          additionalProperties:
+            type: string
+
+    LoadViewResult:
+      description: |
+        Result used when a view is successfully loaded.
+
+
+        The view metadata JSON is returned in the `metadata` field. The corresponding file location of view metadata is returned in the `metadata-location` field.
+        Clients can check whether metadata has changed by comparing metadata locations after the view has been created.
+
+
+        The `config` map returns view-specific configuration for the view's resources, including its HTTP client and FileIO.
+        For example, config may contain a specific FileIO implementation class for the view depending on its underlying storage.

Review Comment:
   Again, we may want to change the example 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java:
##########
@@ -35,17 +35,22 @@
 import org.apache.iceberg.catalog.SupportsNamespaces;
 import org.apache.iceberg.catalog.TableCommit;
 import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.catalog.ViewCatalog;
 import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
 import org.apache.iceberg.exceptions.NoSuchNamespaceException;
 import org.apache.iceberg.hadoop.Configurable;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.view.View;
+import org.apache.iceberg.view.ViewBuilder;
 
-public class RESTCatalog implements Catalog, SupportsNamespaces, Configurable<Object>, Closeable {
+public class RESTCatalog
+    implements Catalog, SupportsNamespaces, Configurable<Object>, Closeable, ViewCatalog {

Review Comment:
   Nit: I'd put `ViewCatalog` just after `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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1630,6 +1990,102 @@ components:
         metadata-log:
           $ref: '#/components/schemas/MetadataLog'
 
+    SQLViewRepresentation:
+      type: object
+      required:
+        - type
+        - sql
+        - dialect
+      properties:
+        type:
+          type: string
+        sql:
+          type: string
+        dialect:
+          type: string
+
+    ViewRepresentation:
+      oneOf:
+        - $ref: '#/components/schemas/SQLViewRepresentation'
+
+    ViewHistoryEntry:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+
+    ViewVersion:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+        - schema-id
+        - summary
+        - representations
+        - default-namespace
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+        schema-id:
+          type: integer

Review Comment:
   @nastra, we need to replace the schema ID and pass -1 here if the schema was just added. The server side may reassign the schema ID so it needs to be replaced with a placeholder.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -971,4 +996,261 @@ public void commitTransaction(SessionContext context, List<TableCommit> commits)
         headers(context),
         ErrorHandlers.tableCommitHandler());
   }
+
+  @Override
+  public List<TableIdentifier> listViews(SessionContext context, Namespace namespace) {
+    checkNamespaceIsValid(namespace);
+
+    ListTablesResponse response =
+        client.get(
+            paths.views(namespace),
+            ListTablesResponse.class,
+            headers(context),
+            ErrorHandlers.namespaceErrorHandler());
+    return response.identifiers();
+  }
+
+  @Override
+  public View loadView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    LoadViewResponse response =
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    ViewMetadata metadata = response.metadata();
+
+    RESTViewOperations ops =
+        new RESTViewOperations(
+            client,
+            paths.view(identifier),
+            session::headers,
+            tableFileIO(context, response.config()),
+            metadata);
+
+    trackFileIO(ops);
+
+    return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+  }
+
+  @Override
+  public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) {
+    return new RESTViewBuilder(context, identifier);
+  }
+
+  @Override
+  public boolean dropView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    try {
+      client.delete(
+          paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler());
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) {
+    checkViewIdentifierIsValid(from);
+    checkViewIdentifierIsValid(to);
+
+    RenameTableRequest request =
+        RenameTableRequest.builder().withSource(from).withDestination(to).build();
+
+    client.post(
+        paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler());
+  }
+
+  private class RESTViewBuilder implements ViewBuilder {
+    private final SessionContext context;
+    private final TableIdentifier identifier;
+    private final Map<String, String> properties = Maps.newHashMap();
+    private final List<ViewRepresentation> representations = Lists.newArrayList();
+    private Namespace defaultNamespace = null;
+    private String defaultCatalog = null;
+    private Schema schema = null;
+    private String location = null;
+
+    private RESTViewBuilder(SessionContext context, TableIdentifier identifier) {
+      checkViewIdentifierIsValid(identifier);
+      this.identifier = identifier;
+      this.context = context;
+    }
+
+    @Override
+    public ViewBuilder withSchema(Schema newSchema) {
+      this.schema = newSchema;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withQuery(String dialect, String sql) {
+      representations.add(
+          ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build());
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultCatalog(String catalog) {
+      this.defaultCatalog = catalog;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withDefaultNamespace(Namespace namespace) {
+      this.defaultNamespace = namespace;
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperties(Map<String, String> newProperties) {
+      this.properties.putAll(newProperties);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withProperty(String key, String value) {
+      this.properties.put(key, value);
+      return this;
+    }
+
+    @Override
+    public ViewBuilder withLocation(String newLocation) {
+      this.location = newLocation;
+      return this;
+    }
+
+    @Override
+    public View create() {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot create view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot create view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot create view without specifying a default namespace");
+
+      ViewVersion viewVersion =
+          ImmutableViewVersion.builder()
+              .versionId(1)
+              .schemaId(schema.schemaId())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .defaultCatalog(defaultCatalog)
+              .timestampMillis(System.currentTimeMillis())
+              .putSummary("operation", "create")
+              .build();
+
+      CreateViewRequest request =
+          ImmutableCreateViewRequest.builder()
+              .name(identifier.name())
+              .location(location)
+              .schema(schema)
+              .viewVersion(viewVersion)
+              .properties(properties)
+              .build();
+
+      LoadViewResponse response =
+          client.post(
+              paths.views(identifier.namespace()),
+              request,
+              LoadViewResponse.class,
+              headers(context),
+              ErrorHandlers.viewErrorHandler());
+
+      return viewFromResponse(response);
+    }
+
+    @Override
+    public View createOrReplace() {
+      try {
+        return replace(loadView());
+      } catch (NoSuchViewException e) {
+        return create();
+      }
+    }
+
+    @Override
+    public View replace() {
+      return replace(loadView());
+    }
+
+    private LoadViewResponse loadView() {
+      return client.get(
+          paths.view(identifier),
+          LoadViewResponse.class,
+          headers(context),
+          ErrorHandlers.viewErrorHandler());
+    }
+
+    private View replace(LoadViewResponse response) {
+      Preconditions.checkState(
+          !representations.isEmpty(), "Cannot replace view without specifying a query");
+      Preconditions.checkState(null != schema, "Cannot replace view without specifying schema");
+      Preconditions.checkState(
+          null != defaultNamespace, "Cannot replace view without specifying a default namespace");
+
+      ViewMetadata metadata = response.metadata();
+
+      int maxVersionId =
+          metadata.versions().stream()
+              .map(ViewVersion::versionId)
+              .max(Integer::compareTo)
+              .orElseGet(metadata::currentVersionId);
+
+      ViewVersion viewVersion =
+          ImmutableViewVersion.builder()
+              .versionId(maxVersionId + 1)
+              .schemaId(schema.schemaId())
+              .addAllRepresentations(representations)
+              .defaultNamespace(defaultNamespace)
+              .defaultCatalog(defaultCatalog)
+              .timestampMillis(System.currentTimeMillis())
+              .putSummary("operation", "replace")
+              .build();
+
+      ViewMetadata.Builder builder =
+          ViewMetadata.buildFrom(metadata)
+              .setProperties(properties)
+              .setCurrentVersion(viewVersion, schema);
+
+      if (null != location) {
+        builder.setLocation(location);
+      }
+
+      ViewMetadata replacement = builder.build();
+
+      UpdateTableRequest request =
+          UpdateTableRequest.create(identifier, ImmutableList.of(), replacement.changes());

Review Comment:
   makes sense, I've updated that



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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1860,6 +2320,26 @@ components:
               items:
                 type: string
 
+    AddViewVersionUpdate:
+      allOf:
+        - $ref: '#/components/schemas/BaseUpdate'
+        - type: object
+          required:
+            - view-version
+          properties:
+            view-version:
+              $ref: '#/components/schemas/ViewVersion'
+
+    SetCurrentViewVersionUpdate:
+      allOf:
+        - $ref: '#/components/schemas/BaseUpdate'
+        - type: object
+          required:
+            - view-version-id

Review Comment:
   agreed, I've added a description to indicate that



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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -89,11 +94,13 @@ public class RESTCatalogAdapter implements RESTClient {
 
   private final Catalog catalog;
   private final SupportsNamespaces asNamespaceCatalog;
+  private final ViewCatalog viewCatalog;

Review Comment:
   Nit: `asViewCatalog` to match the `asNamespaceCatalog` name? That way it's more clear that this isn't a completely different view 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest;
+import org.apache.iceberg.rest.responses.LoadViewResponse;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.iceberg.view.ViewOperations;
+
+class RESTViewOperations implements ViewOperations {
+
+  private final RESTClient client;
+  private final String path;
+  private final Supplier<Map<String, String>> headers;
+  private final FileIO io;
+  private ViewMetadata current;
+
+  RESTViewOperations(
+      RESTClient client,
+      String path,
+      Supplier<Map<String, String>> headers,
+      FileIO io,
+      ViewMetadata current) {
+    Preconditions.checkArgument(null != current, "Invalid view metadata: null");
+    this.client = client;
+    this.path = path;
+    this.headers = headers;
+    this.io = io;
+    this.current = current;
+  }
+
+  @Override
+  public ViewMetadata current() {
+    return current;
+  }
+
+  @Override
+  public ViewMetadata refresh() {
+    return updateCurrentMetadata(
+        client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler()));
+  }
+
+  @Override
+  public void commit(ViewMetadata base, ViewMetadata metadata) {
+    // this is only used for replacing view metadata
+    Preconditions.checkState(base != null, "Invalid base metadata: null");
+
+    UpdateTableRequest request =
+        UpdateTableRequest.create(null, ImmutableList.of(), metadata.changes());

Review Comment:
   using an assertion on the View UUID was the plan (I just haven't pushed the code yet). I wanted to first slightly refactor the requirements in https://github.com/apache/iceberg/pull/8831



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -247,8 +247,9 @@ public void createViewErrorCases() {
                     .withQuery(trino.dialect(), trino.sql())
                     .withQuery(trino.dialect(), trino.sql())
                     .create())
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Invalid view version: Cannot add multiple queries for dialect trino");
+        .isInstanceOf(Exception.class)

Review Comment:
   Minor: I think I pointed this out before and may have forgotten the answer, but shouldn't this be a consistent exception class every time?



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
api/src/main/java/org/apache/iceberg/catalog/ViewSessionCatalog.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.catalog;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchViewException;
+import org.apache.iceberg.view.View;
+import org.apache.iceberg.view.ViewBuilder;
+
+/** A session Catalog API for view create, drop, and load operations. */
+public interface ViewSessionCatalog {
+
+  /**
+   * Return the name for this catalog.
+   *
+   * @return this catalog's name
+   */
+  String name();
+
+  /**
+   * Return all the identifiers under this namespace.
+   *
+   * @param namespace a namespace
+   * @return a list of identifiers for views
+   * @throws NoSuchNamespaceException if the namespace is not found
+   */
+  List<TableIdentifier> listViews(SessionCatalog.SessionContext context, Namespace namespace);
+
+  /**
+   * Load a view.
+   *
+   * @param identifier a view identifier
+   * @return instance of {@link View} implementation referred by the identifier
+   * @throws NoSuchViewException if the view does not exist
+   */
+  View loadView(SessionCatalog.SessionContext context, TableIdentifier identifier);
+
+  /**
+   * Check whether view exists.
+   *
+   * @param identifier a view identifier
+   * @return true if the view exists, false otherwise
+   */
+  default boolean viewExists(SessionCatalog.SessionContext context, TableIdentifier identifier) {
+    try {
+      loadView(context, identifier);
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  /**
+   * Instantiate a builder to create or replace a SQL view.
+   *
+   * @param identifier a view identifier
+   * @return a view builder
+   */
+  ViewBuilder buildView(SessionCatalog.SessionContext context, TableIdentifier identifier);
+
+  /**
+   * Drop a view.
+   *
+   * @param identifier a view identifier
+   * @return true if the view was dropped, false if the view did not exist
+   */
+  boolean dropView(SessionCatalog.SessionContext context, TableIdentifier identifier);
+
+  /**
+   * Rename a view.
+   *
+   * @param from identifier of the view to rename
+   * @param to new view identifier
+   * @throws NoSuchViewException if the "from" view does not exist
+   * @throws AlreadyExistsException if the "to" view already exists
+   */
+  void renameView(SessionCatalog.SessionContext context, TableIdentifier from, TableIdentifier to);
+
+  /**
+   * Invalidate cached view metadata from current catalog.
+   *
+   * <p>If the view is already loaded or cached, drop cached data. If the view does not exist or is
+   * not cached, do nothing.
+   *
+   * @param identifier a view identifier
+   */
+  default void invalidateView(SessionCatalog.SessionContext context, TableIdentifier identifier) {}
+
+  /**
+   * Initialize a view catalog given a custom name and a map of catalog properties.
+   *
+   * <p>A custom view catalog implementation must have a no-arg constructor. A compute engine like
+   * Spark or Flink will first initialize the catalog without any arguments, and then call this
+   * method to complete catalog initialization with properties passed into the engine.
+   *
+   * @param name a custom name for the catalog
+   * @param properties catalog properties
+   */
+  default void initialize(String name, Map<String, String> properties) {}

Review Comment:
   There's no need for a default here since this is a new interface. It's fine for `invalidateView` because most catalogs won't cache, but `initialize` is something we want implemented. This should also avoid issues with conflicting defaults.



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();

Review Comment:
   This should not pass full view metadata. It should pass the parts of view metadata that are used to create a view, just like tables.
   
   Otherwise, the contract for this route is confusing and loses data. The code below only uses `currentVersion`, but what if `currentVersion` is not the only version?



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1630,6 +1990,102 @@ components:
         metadata-log:
           $ref: '#/components/schemas/MetadataLog'
 
+    SQLViewRepresentation:
+      type: object
+      required:
+        - type
+        - sql
+        - dialect
+      properties:
+        type:
+          type: string
+        sql:
+          type: string
+        dialect:
+          type: string
+
+    ViewRepresentation:
+      oneOf:
+        - $ref: '#/components/schemas/SQLViewRepresentation'
+
+    ViewHistoryEntry:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+
+    ViewVersion:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+        - schema-id
+        - summary
+        - representations
+        - default-namespace
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+        schema-id:
+          type: integer

Review Comment:
   This can be -1 to signal that it should be the last added schema?



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1630,6 +1990,102 @@ components:
         metadata-log:
           $ref: '#/components/schemas/MetadataLog'
 
+    SQLViewRepresentation:
+      type: object
+      required:
+        - type
+        - sql
+        - dialect
+      properties:
+        type:
+          type: string
+        sql:
+          type: string
+        dialect:
+          type: string
+
+    ViewRepresentation:
+      oneOf:
+        - $ref: '#/components/schemas/SQLViewRepresentation'
+
+    ViewHistoryEntry:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+
+    ViewVersion:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+        - schema-id
+        - summary
+        - representations
+        - default-namespace

Review Comment:
   default catalog isn't a required field in view version, hence it's not listed in the `required` section 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -2033,6 +2537,68 @@ components:
         metadata-location:
           type: string
 
+    CreateViewRequest:
+      type: object
+      required:
+        - name
+        - schema
+        - view-version
+        - properties
+      properties:
+        name:
+          type: string
+        location:
+          type: string
+        schema:
+          $ref: '#/components/schemas/Schema'
+        view-version:
+          $ref: '#/components/schemas/ViewVersion'
+        properties:
+          type: object
+          additionalProperties:
+            type: string
+
+    LoadViewResult:
+      description: |
+        Result used when a view is successfully loaded.
+
+
+        The view metadata JSON is returned in the `metadata` field. The corresponding file location of view metadata is returned in the `metadata-location` field.

Review Comment:
   I've checked and it's actually shorter than the one for `LoadTableResult`, so I've just tried to align everything similar to how other result descriptions are structured



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest;
+import org.apache.iceberg.rest.responses.LoadViewResponse;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.iceberg.view.ViewOperations;
+
+class RESTViewOperations implements ViewOperations {
+
+  private final RESTClient client;
+  private final String path;
+  private final Supplier<Map<String, String>> headers;
+  private final FileIO io;
+  private ViewMetadata current;
+
+  RESTViewOperations(
+      RESTClient client,
+      String path,
+      Supplier<Map<String, String>> headers,
+      FileIO io,
+      ViewMetadata current) {
+    Preconditions.checkArgument(null != current, "Invalid view metadata: null");
+    this.client = client;
+    this.path = path;
+    this.headers = headers;
+    this.io = io;
+    this.current = current;
+  }
+
+  @Override
+  public ViewMetadata current() {
+    return current;
+  }
+
+  @Override
+  public ViewMetadata refresh() {
+    return updateCurrentMetadata(
+        client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler()));
+  }
+
+  @Override
+  public void commit(ViewMetadata base, ViewMetadata metadata) {
+    // this is only used for replacing view metadata
+    Preconditions.checkState(base != null, "Invalid base metadata: null");
+
+    UpdateTableRequest request =
+        UpdateTableRequest.create(null, ImmutableList.of(), metadata.changes());

Review Comment:
   I've added `AssertTableUUID` here directly for now. After https://github.com/apache/iceberg/pull/8831 I was planning to add a `UpdateRequirements.forReplaceView(...)` method incl tests, similar to how we do for tables. I think this should unblock this PR for now and we can do the refactoring independently



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -30,8 +30,10 @@
 import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.view.View;
+import org.apache.iceberg.view.ViewBuilder;
 
-public abstract class BaseSessionCatalog implements SessionCatalog {
+public abstract class BaseSessionCatalog implements SessionCatalog, ViewSessionCatalog {

Review Comment:
   Yes, I think we should. And let's track down why RevAPI didn't complain! That seems dangerous.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.rest.requests.UpdateTableRequest;
+import org.apache.iceberg.rest.responses.LoadViewResponse;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.iceberg.view.ViewOperations;
+
+class RESTViewOperations implements ViewOperations {

Review Comment:
   This looks correct to me. Nice that we don't need to handle create changes because we don't have transactions.
   
   We may need to add view creation to a multi-table transaction later, though.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +386,128 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  private static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");

Review Comment:
   The Table methods use `IllegalStateException` with `Cannot wrap catalog that does not produce BaseTable`. Can you make the behavior similar?



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));
+    View view = viewBuilder.create();
+
+    return viewResponse(view);
+  }
+
+  private static LoadViewResponse viewResponse(View view) {
+    ViewMetadata metadata = baseView(view).operations().current();
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    View view = catalog.loadView(viewIdentifier);
+    return viewResponse(view);
+  }
+
+  public static LoadViewResponse updateView(
+      ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) {
+    View view = catalog.loadView(ident);
+    ViewMetadata metadata = commit(baseView(view).operations(), request);
+
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static void renameView(ViewCatalog catalog, RenameTableRequest request) {
+    catalog.renameView(request.source(), request.destination());
+  }
+
+  public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    boolean dropped = catalog.dropView(viewIdentifier);
+    if (!dropped) {
+      throw new NoSuchViewException("View does not exist: %s", viewIdentifier);
+    }
+  }
+
+  static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
+    AtomicBoolean isRetry = new AtomicBoolean(false);
+    try {
+      Tasks.foreach(ops)
+          .retry(COMMIT_NUM_RETRIES_DEFAULT)
+          .exponentialBackoff(
+              COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
+              2.0 /* exponential */)
+          .onlyRetryOn(CommitFailedException.class)
+          .run(
+              taskOps -> {
+                ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current();
+                isRetry.set(true);
+
+                // apply changes
+                ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base);
+                request.updates().forEach(update -> update.applyTo(metadataBuilder));

Review Comment:
   Why does this skip validating requirements?



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequest.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.apache.iceberg.view.ViewMetadata;
+import org.immutables.value.Value;
+
+@Value.Immutable
+public interface CreateViewRequest extends RESTRequest {
+  String name();
+
+  ViewMetadata metadata();

Review Comment:
   As I noted above, this needs to send individual view metadata parts, rather than a complete ViewMetadata.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1014,6 +1014,357 @@ paths:
                 }
               }
 
+  /v1/{prefix}/namespaces/{namespace}/views:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    get:
+      tags:
+        - Catalog API
+      summary: List all view identifiers underneath a given namespace
+      description: Return all view identifiers under this namespace
+      operationId: listViews
+      responses:
+        200:
+          $ref: '#/components/responses/ListTablesResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Create a view in the given namespace
+      description:
+        Create a view in the given namespace.
+      operationId: createView
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CreateViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        409:
+          description: Conflict - The view already exists
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceAlreadyExists:
+                  $ref: '#/components/examples/ViewAlreadyExistsError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/namespaces/{namespace}/views/{view}:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+      - $ref: '#/components/parameters/view'
+
+    get:
+      tags:
+        - Catalog API
+      summary: Load a view from the catalog
+      operationId: loadView
+      description:
+        Load a view from the catalog.
+
+
+        The response contains both configuration and table metadata. The configuration, if non-empty is used
+        as additional configuration for the view that overrides catalog configuration. For example, this
+        configuration may change the FileIO implementation to be used for the view.
+
+
+        The response also contains the view's full metadata, matching the view metadata JSON file.
+
+
+        The catalog configuration may contain credentials that should be used for subsequent requests for the
+        view. The configuration key "token" is used to pass an access token to be used as a bearer token
+        for view requests. Otherwise, a token may be passed using a RFC 8693 token type as a configuration
+        key. For example, "urn:ietf:params:oauth:token-type:jwt=<JWT-token>".
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToLoadDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Replace a view
+      operationId: replaceView
+      description:
+        Commit updates to a view.
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToUpdateDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        409:
+          description:
+            Conflict - CommitFailedException. The client may retry.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        500:
+          description:
+            An unknown server-side problem occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Internal Server Error",
+                  "type": "CommitStateUnknownException",
+                  "code": 500
+                }
+              }
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        502:
+          description:
+            A gateway or proxy received an invalid response from the upstream server; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Invalid response from the upstream server",
+                  "type": "CommitStateUnknownException",
+                  "code": 502
+                }
+              }
+        504:
+          description:
+            A server-side gateway timeout occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Gateway timed out during commit",
+                  "type": "CommitStateUnknownException",
+                  "code": 504
+                }
+              }
+        5XX:
+          description:
+            A server-side problem that might not be addressable on the client.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Bad Gateway",
+                  "type": "InternalServerError",
+                  "code": 502
+                }
+              }
+
+    delete:
+      tags:
+        - Catalog API
+      summary: Drop a view from the catalog
+      operationId: dropView
+      description: Remove a view from the catalog
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to drop does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToDeleteDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    head:
+      tags:
+        - Catalog API
+      summary: Check if a view exists
+      operationId: viewExists
+      description:
+        Check if a view exists within a given namespace. This request does not return a response body.
+      responses:
+        200:
+          description: OK - View Exists
+        400:
+          description: Bad Request
+        401:
+          description: Unauthorized
+        404:
+          description: Not Found
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/views/rename:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Rename a view from its current name to a new name
+      description:
+        Rename a view from one identifier to another. It's valid to move a view
+        across namespaces, but the server implementation is not required to support it.
+      operationId: renameView
+      requestBody:
+        description: Current view identifier to rename and new view identifier to rename to
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/RenameTableRequest'
+            examples:
+              RenameViewSameNamespace:
+                $ref: '#/components/examples/RenameViewSameNamespace'
+        required: true
+      responses:
+        200:
+          description: OK
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found
+            - NoSuchViewException, view to rename does not exist
+            - NoSuchNamespaceException, The target namespace of the new table identifier does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToRenameDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+                NamespaceToRenameToDoesNotExist:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        406:
+          $ref: '#/components/responses/UnsupportedOperationResponse'
+        409:
+          description: Conflict - The target view identifier to rename to already exists

Review Comment:
   we do have `renameViewTargetAlreadyExistsAsTable()`, but we don't have a version of it that would try and rename a table that exists already as a view. I'll add 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/UpdateRequirement.java:
##########
@@ -20,11 +20,17 @@
 
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.view.ViewMetadata;
 
 /** Represents a requirement for a {@link MetadataUpdate} */
 public interface UpdateRequirement {
   void validate(TableMetadata base);
 
+  default void validate(ViewMetadata base) {
+    throw new UnsupportedOperationException(

Review Comment:
   @nastra, looks like this didn't get updated.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -247,8 +247,9 @@ public void createViewErrorCases() {
                     .withQuery(trino.dialect(), trino.sql())
                     .withQuery(trino.dialect(), trino.sql())
                     .create())
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Invalid view version: Cannot add multiple queries for dialect trino");
+        .isInstanceOf(Exception.class)

Review Comment:
   I remember I replied to this but somehow can't find where it was.
   The issue with the REST version is unfortunately that it fails with `org.apache.iceberg.exceptions.BadRequestException: Malformed request: Invalid view version: Cannot add multiple queries for dialect trino` because we're re-applying all changes to the Builder in https://github.com/apache/iceberg/blob/f076b4d15822a7b4f9a0d7a0c65ce62682bff109/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java#L424-L429
   
   That IAE is then converted to a `BadRequestException` in https://github.com/apache/iceberg/blob/ace0b13aa733c750ea8d47adb6a1c27950494ba8/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java#L204-L205
   
   So I think what we could do is to have
   ```
   if (IllegalArgumentException.class.getSimpleName().equals(error.type())) {
           throw new IllegalArgumentException(error.message());
   }
   ```
   in the `DefaultErrorHandler`. Thoughts?



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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -247,8 +247,9 @@ public void createViewErrorCases() {
                     .withQuery(trino.dialect(), trino.sql())
                     .withQuery(trino.dialect(), trino.sql())
                     .create())
-        .isInstanceOf(IllegalArgumentException.class)
-        .hasMessage("Invalid view version: Cannot add multiple queries for dialect trino");
+        .isInstanceOf(Exception.class)

Review Comment:
   I remember I replied to this but somehow can't find where it was.
   The issue with the REST version is unfortunately that it fails with `org.apache.iceberg.exceptions.BadRequestException: Malformed request: Invalid view version: Cannot add multiple queries for dialect trino` because we're re-applying all changes to the Builder in `CatalogHandlers.createView(...)`: https://github.com/apache/iceberg/blob/f076b4d15822a7b4f9a0d7a0c65ce62682bff109/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java#L424-L429
   
   That IAE is then converted to a `BadRequestException` in https://github.com/apache/iceberg/blob/ace0b13aa733c750ea8d47adb6a1c27950494ba8/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java#L204-L205
   
   So I think what we could do is to have
   ```
   if (IllegalArgumentException.class.getSimpleName().equals(error.type())) {
           throw new IllegalArgumentException(error.message());
   }
   ```
   in the `DefaultErrorHandler`. Thoughts?



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

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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/view/ViewMetadata.java:
##########
@@ -257,8 +258,8 @@ public Builder addVersion(ViewVersion version) {
       return this;
     }
 
-    private int addVersionInternal(ViewVersion version) {
-      int newVersionId = reuseOrCreateNewViewVersionId(version);
+    private int addVersionInternal(ViewVersion newVersion) {

Review Comment:
   I've renamed this parameter to make the code slightly easier to understand, because we have a few cases in this method where we re-assign either the schema id or the view version id



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1630,6 +1990,102 @@ components:
         metadata-log:
           $ref: '#/components/schemas/MetadataLog'
 
+    SQLViewRepresentation:
+      type: object
+      required:
+        - type
+        - sql
+        - dialect
+      properties:
+        type:
+          type: string
+        sql:
+          type: string
+        dialect:
+          type: string
+
+    ViewRepresentation:
+      oneOf:
+        - $ref: '#/components/schemas/SQLViewRepresentation'
+
+    ViewHistoryEntry:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+
+    ViewVersion:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+        - schema-id
+        - summary
+        - representations
+        - default-namespace
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+        schema-id:
+          type: integer

Review Comment:
   this should be done now with the last commit



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -1534,4 +1535,90 @@ public void updateViewLocationConflict() {
         .isInstanceOf(NoSuchViewException.class)
         .hasMessageContaining("View does not exist: ns.view");
   }
+
+  @Test
+  public void concurrentReplaceViewVersion() {
+    TableIdentifier identifier = TableIdentifier.of("ns", "view");
+
+    if (requiresNamespaceCreate()) {
+      catalog().createNamespace(identifier.namespace());
+    }
+
+    assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse();
+
+    View view =
+        catalog()
+            .buildView(identifier)
+            .withSchema(SCHEMA)
+            .withDefaultNamespace(identifier.namespace())
+            .withQuery("trino", "select * from ns.tbl")
+            .create();
+
+    assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue();
+
+    ReplaceViewVersion replaceViewVersion =
+        view.replaceVersion()
+            .withQuery("trino", "select count(*) from ns.tbl")
+            .withSchema(SCHEMA)
+            .withDefaultNamespace(identifier.namespace());
+
+    ReplaceViewVersion replaceViewVersionConcurrent =
+        view.replaceVersion()
+            .withQuery("spark", "select count(*) from ns.tbl")
+            .withSchema(OTHER_SCHEMA)
+            .withDefaultNamespace(identifier.namespace());

Review Comment:
   yes this was a bit difficult to test a true concurrent commit due to the refresh happening in `ViewVersionReplace`. I've switched things up to how you suggested and the test passes for the REST catalog and fails otherwise.
   Thanks for those suggestions.



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

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 #7913: Core: Add View support for REST catalog

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


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

Review Comment:
   I talked with @danielcweeks about this last week and I think it is going to cause unnecessary conflicts to go back to the metadata level. I think we should use the change based approach like the REST table code, with `AddViewVersion` and `SetCurrentViewVersion` 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));
+    View view = viewBuilder.create();
+
+    return viewResponse(view);
+  }
+
+  private static LoadViewResponse viewResponse(View view) {
+    ViewMetadata metadata = baseView(view).operations().current();
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    View view = catalog.loadView(viewIdentifier);
+    return viewResponse(view);
+  }
+
+  public static LoadViewResponse updateView(
+      ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) {
+    View view = catalog.loadView(ident);
+    ViewMetadata metadata = commit(baseView(view).operations(), request);
+
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static void renameView(ViewCatalog catalog, RenameTableRequest request) {
+    catalog.renameView(request.source(), request.destination());
+  }
+
+  public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    boolean dropped = catalog.dropView(viewIdentifier);
+    if (!dropped) {
+      throw new NoSuchViewException("View does not exist: %s", viewIdentifier);
+    }
+  }
+
+  static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
+    AtomicBoolean isRetry = new AtomicBoolean(false);
+    try {
+      Tasks.foreach(ops)
+          .retry(COMMIT_NUM_RETRIES_DEFAULT)
+          .exponentialBackoff(
+              COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
+              2.0 /* exponential */)
+          .onlyRetryOn(CommitFailedException.class)
+          .run(
+              taskOps -> {
+                ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current();
+                isRetry.set(true);
+
+                // apply changes
+                ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base);
+                request.updates().forEach(update -> update.applyTo(metadataBuilder));

Review Comment:
   I think we will need to support the AssertUUID requirement right? It also seems good to run them even if we expect there to be none. Otherwise, when we add a requirement we'll have to go back and find out why it doesn't actually work. And similarly, if someone sends a requirement that is not allowed for views (but can be parsed for tables) we need to attempt to run it so that we will throw the exception that it isn't supported.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -2033,6 +2537,68 @@ components:
         metadata-location:
           type: string
 
+    CreateViewRequest:
+      type: object
+      required:
+        - name
+        - schema
+        - view-version
+        - properties
+      properties:
+        name:
+          type: string
+        location:
+          type: string
+        schema:
+          $ref: '#/components/schemas/Schema'
+        view-version:
+          $ref: '#/components/schemas/ViewVersion'
+        properties:
+          type: object
+          additionalProperties:
+            type: string
+
+    LoadViewResult:
+      description: |
+        Result used when a view is successfully loaded.
+
+
+        The view metadata JSON is returned in the `metadata` field. The corresponding file location of view metadata is returned in the `metadata-location` field.
+        Clients can check whether metadata has changed by comparing metadata locations after the view has been created.
+
+
+        The `config` map returns view-specific configuration for the view's resources, including its HTTP client and FileIO.
+        For example, config may contain a specific FileIO implementation class for the view depending on its underlying storage.
+
+
+        The following configurations should be respected by clients:
+
+        ## General Configurations
+
+        - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled 
+
+        ## AWS Configurations
+
+        The following configurations should be respected when working with views stored in AWS S3
+         - `client.region`: region to configure client for making requests to AWS
+         - `s3.access-key-id`: id for for credentials that provide access to the data in S3
+         - `s3.secret-access-key`: secret for credentials that provide access to data in S3 
+         - `s3.session-token`: if present, this value should be used for as the session token 
+         - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification

Review Comment:
   agreed, now that we don't need `FileIO` in the view ops class, we can remove these



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));
+    View view = viewBuilder.create();
+
+    return viewResponse(view);
+  }
+
+  private static LoadViewResponse viewResponse(View view) {
+    ViewMetadata metadata = baseView(view).operations().current();
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    View view = catalog.loadView(viewIdentifier);
+    return viewResponse(view);
+  }
+
+  public static LoadViewResponse updateView(
+      ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) {
+    View view = catalog.loadView(ident);
+    ViewMetadata metadata = commit(baseView(view).operations(), request);
+
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static void renameView(ViewCatalog catalog, RenameTableRequest request) {
+    catalog.renameView(request.source(), request.destination());
+  }
+
+  public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    boolean dropped = catalog.dropView(viewIdentifier);
+    if (!dropped) {
+      throw new NoSuchViewException("View does not exist: %s", viewIdentifier);
+    }
+  }
+
+  static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
+    AtomicBoolean isRetry = new AtomicBoolean(false);
+    try {
+      Tasks.foreach(ops)
+          .retry(COMMIT_NUM_RETRIES_DEFAULT)
+          .exponentialBackoff(
+              COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
+              2.0 /* exponential */)
+          .onlyRetryOn(CommitFailedException.class)
+          .run(
+              taskOps -> {
+                ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current();
+                isRetry.set(true);
+
+                // apply changes
+                ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base);
+                request.updates().forEach(update -> update.applyTo(metadataBuilder));

Review Comment:
   I'd like to get this working with `AssertTableUUID`, independent of adding `AssertUUID`. It doesn't bother me to use the table classes temporarily.
   
   I don't think we need `AssertCurrentSchemaID`.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -1444,9 +1522,14 @@ public void updateViewLocationConflict() {
     catalog().dropView(identifier);
     assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse();
 
+    String expectedMessage =
+        catalog() instanceof RESTCatalog ? "View does not exist: ns.view" : "Cannot commit";
+    Class<?> expectedException =
+        catalog() instanceof RESTCatalog ? NoSuchViewException.class : CommitFailedException.class;

Review Comment:
   done in https://github.com/apache/iceberg/pull/8865



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1056,6 +1056,357 @@ paths:
                 }
               }
 
+  /v1/{prefix}/namespaces/{namespace}/views:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+
+    get:
+      tags:
+        - Catalog API
+      summary: List all view identifiers underneath a given namespace
+      description: Return all view identifiers under this namespace
+      operationId: listViews
+      responses:
+        200:
+          $ref: '#/components/responses/ListTablesResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Create a view in the given namespace
+      description:
+        Create a view in the given namespace.
+      operationId: createView
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CreateViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description: Not Found - The namespace specified does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceNotFound:
+                  $ref: '#/components/examples/NoSuchNamespaceError'
+        409:
+          description: Conflict - The view already exists
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                NamespaceAlreadyExists:
+                  $ref: '#/components/examples/ViewAlreadyExistsError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+  /v1/{prefix}/namespaces/{namespace}/views/{view}:
+    parameters:
+      - $ref: '#/components/parameters/prefix'
+      - $ref: '#/components/parameters/namespace'
+      - $ref: '#/components/parameters/view'
+
+    get:
+      tags:
+        - Catalog API
+      summary: Load a view from the catalog
+      operationId: loadView
+      description:
+        Load a view from the catalog.
+
+
+        The response contains both configuration and view metadata. The configuration, if non-empty is used
+        as additional configuration for the view that overrides catalog configuration. For example, this
+        configuration may change the FileIO implementation to be used for the view.
+
+
+        The response also contains the view's full metadata, matching the view metadata JSON file.
+
+
+        The catalog configuration may contain credentials that should be used for subsequent requests for the
+        view. The configuration key "token" is used to pass an access token to be used as a bearer token
+        for view requests. Otherwise, a token may be passed using a RFC 8693 token type as a configuration
+        key. For example, "urn:ietf:params:oauth:token-type:jwt=<JWT-token>".
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToLoadDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    post:
+      tags:
+        - Catalog API
+      summary: Replace a view
+      operationId: replaceView
+      description:
+        Commit updates to a view.
+      requestBody:
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/CommitViewRequest'
+      responses:
+        200:
+          $ref: '#/components/responses/LoadViewResponse'
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to load does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToUpdateDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        409:
+          description:
+            Conflict - CommitFailedException. The client may retry.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        500:
+          description:
+            An unknown server-side problem occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Internal Server Error",
+                  "type": "CommitStateUnknownException",
+                  "code": 500
+                }
+              }
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        502:
+          description:
+            A gateway or proxy received an invalid response from the upstream server; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Invalid response from the upstream server",
+                  "type": "CommitStateUnknownException",
+                  "code": 502
+                }
+              }
+        504:
+          description:
+            A server-side gateway timeout occurred; the commit state is unknown.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Gateway timed out during commit",
+                  "type": "CommitStateUnknownException",
+                  "code": 504
+                }
+              }
+        5XX:
+          description:
+            A server-side problem that might not be addressable on the client.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              example: {
+                "error": {
+                  "message": "Bad Gateway",
+                  "type": "InternalServerError",
+                  "code": 502
+                }
+              }
+
+    delete:
+      tags:
+        - Catalog API
+      summary: Drop a view from the catalog
+      operationId: dropView
+      description: Remove a view from the catalog
+      responses:
+        204:
+          description: Success, no content
+        400:
+          $ref: '#/components/responses/BadRequestErrorResponse'
+        401:
+          $ref: '#/components/responses/UnauthorizedResponse'
+        403:
+          $ref: '#/components/responses/ForbiddenResponse'
+        404:
+          description:
+            Not Found - NoSuchViewException, view to drop does not exist
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/ErrorModel'
+              examples:
+                ViewToDeleteDoesNotExist:
+                  $ref: '#/components/examples/NoSuchViewError'
+        419:
+          $ref: '#/components/responses/AuthenticationTimeoutResponse'
+        503:
+          $ref: '#/components/responses/ServiceUnavailableResponse'
+        5XX:
+          $ref: '#/components/responses/ServerErrorResponse'
+
+    head:
+      tags:
+        - Catalog API
+      summary: Check if a view exists
+      operationId: viewExists
+      description:
+        Check if a view exists within a given namespace. This request does not return a response body.
+      responses:
+        200:

Review Comment:
   This is probably a more general question than we need to worry about here. After this PR, we should revisit 200/204 responses to make the protocol more consistent.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -2033,6 +2537,68 @@ components:
         metadata-location:
           type: string
 
+    CreateViewRequest:
+      type: object
+      required:
+        - name
+        - schema
+        - view-version
+        - properties
+      properties:
+        name:
+          type: string
+        location:
+          type: string
+        schema:
+          $ref: '#/components/schemas/Schema'
+        view-version:
+          $ref: '#/components/schemas/ViewVersion'
+        properties:
+          type: object
+          additionalProperties:
+            type: string
+
+    LoadViewResult:
+      description: |
+        Result used when a view is successfully loaded.
+
+
+        The view metadata JSON is returned in the `metadata` field. The corresponding file location of view metadata is returned in the `metadata-location` field.
+        Clients can check whether metadata has changed by comparing metadata locations after the view has been created.
+
+
+        The `config` map returns view-specific configuration for the view's resources, including its HTTP client and FileIO.
+        For example, config may contain a specific FileIO implementation class for the view depending on its underlying storage.
+
+
+        The following configurations should be respected by clients:
+
+        ## General Configurations
+
+        - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled 
+
+        ## AWS Configurations
+
+        The following configurations should be respected when working with views stored in AWS S3
+         - `client.region`: region to configure client for making requests to AWS
+         - `s3.access-key-id`: id for for credentials that provide access to the data in S3
+         - `s3.secret-access-key`: secret for credentials that provide access to data in S3 
+         - `s3.session-token`: if present, this value should be used for as the session token 
+         - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification

Review Comment:
   I don't think these AWS configurations are valid because we don't initialize a `FileIO` for the view when using 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java:
##########
@@ -1534,4 +1535,90 @@ public void updateViewLocationConflict() {
         .isInstanceOf(NoSuchViewException.class)
         .hasMessageContaining("View does not exist: ns.view");
   }
+
+  @Test
+  public void concurrentReplaceViewVersion() {
+    TableIdentifier identifier = TableIdentifier.of("ns", "view");
+
+    if (requiresNamespaceCreate()) {
+      catalog().createNamespace(identifier.namespace());
+    }
+
+    assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse();
+
+    View view =
+        catalog()
+            .buildView(identifier)
+            .withSchema(SCHEMA)
+            .withDefaultNamespace(identifier.namespace())
+            .withQuery("trino", "select * from ns.tbl")
+            .create();
+
+    assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue();
+
+    ReplaceViewVersion replaceViewVersion =
+        view.replaceVersion()
+            .withQuery("trino", "select count(*) from ns.tbl")
+            .withSchema(SCHEMA)
+            .withDefaultNamespace(identifier.namespace());
+
+    ReplaceViewVersion replaceViewVersionConcurrent =
+        view.replaceVersion()
+            .withQuery("spark", "select count(*) from ns.tbl")
+            .withSchema(OTHER_SCHEMA)
+            .withDefaultNamespace(identifier.namespace());

Review Comment:
   This doesn't actually simulate concurrent commits. This uses the same `view` instance and the `internalApply` method calls refresh. So what's actually happening is that the `ViewVersionReplace` is basing its changes on the correct metadata. That's why in the table version of similar tests, we actually load two separate instance of the table. That way the instance isn't up-to-date when the changes are created.
   
   In addition, because `internalApply` calls refresh, even if you were to use a separate `view` instance it will still update itself before creating the changes.
   
   I think to fix this, we should make the `internalApply` method package-private so that we can call it. Then to commit you can call the `ViewOperations` directly:
   
   ```java
     trinoView = catalog.loadView(identifier);
     ViewMetadata trinoUpdate = trinoView
         .replaceVersion()
         ...
         .internalApply();
   
     ViewOperations trinoOps = ((BaseView) trinoView).operations();
     trinoOps.commit(trinoOps.current(), trinoUpdate);
   ```
   
   That will exercise the server-side retry case, I think.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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

   Thanks @nastra! Great to have this in.


-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/catalog/BaseSessionCatalog.java:
##########
@@ -30,8 +30,10 @@
 import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.view.View;
+import org.apache.iceberg.view.ViewBuilder;
 
-public abstract class BaseSessionCatalog implements SessionCatalog {
+public abstract class BaseSessionCatalog implements SessionCatalog, ViewSessionCatalog {

Review Comment:
   Doesn't adding `ViewSessionCatalog` here break compatibility with existing implementations? It will force them to implement the view methods. I'm surprised this wasn't picked up by revapi.



-- 
This is an automated message from the 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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {

Review Comment:
   Why is this public?



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

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 #7913: Core: Add View support for REST catalog

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -971,4 +996,250 @@ public void commitTransaction(SessionContext context, List<TableCommit> commits)
         headers(context),
         ErrorHandlers.tableCommitHandler());
   }
+
+  @Override
+  public List<TableIdentifier> listViews(SessionContext context, Namespace namespace) {
+    checkNamespaceIsValid(namespace);
+
+    ListTablesResponse response =
+        client.get(
+            paths.views(namespace),
+            ListTablesResponse.class,
+            headers(context),
+            ErrorHandlers.namespaceErrorHandler());
+    return response.identifiers();
+  }
+
+  @Override
+  public View loadView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    LoadViewResponse response =
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    ViewMetadata metadata = response.metadata();
+
+    RESTViewOperations ops =
+        new RESTViewOperations(
+            client,
+            paths.view(identifier),
+            session::headers,
+            tableFileIO(context, response.config()),
+            metadata);
+
+    trackFileIO(ops);
+
+    return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+  }
+
+  @Override
+  public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) {
+    return new RESTViewBuilder(context, identifier);
+  }
+
+  @Override
+  public boolean dropView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    try {
+      client.delete(
+          paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler());
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) {
+    checkViewIdentifierIsValid(from);
+    checkViewIdentifierIsValid(to);
+
+    RenameTableRequest request =
+        RenameTableRequest.builder().withSource(from).withDestination(to).build();
+
+    client.post(
+        paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler());
+  }
+
+  private class RESTViewBuilder implements ViewBuilder {
+    private final SessionContext context;
+    private final TableIdentifier identifier;
+    private final ImmutableViewVersion.Builder viewVersionBuilder = ImmutableViewVersion.builder();

Review Comment:
   Looks like this needs to be updated to match the other version builder's structure.



##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -971,4 +996,250 @@ public void commitTransaction(SessionContext context, List<TableCommit> commits)
         headers(context),
         ErrorHandlers.tableCommitHandler());
   }
+
+  @Override
+  public List<TableIdentifier> listViews(SessionContext context, Namespace namespace) {
+    checkNamespaceIsValid(namespace);
+
+    ListTablesResponse response =
+        client.get(
+            paths.views(namespace),
+            ListTablesResponse.class,
+            headers(context),
+            ErrorHandlers.namespaceErrorHandler());
+    return response.identifiers();
+  }
+
+  @Override
+  public View loadView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    LoadViewResponse response =
+        client.get(
+            paths.view(identifier),
+            LoadViewResponse.class,
+            headers(context),
+            ErrorHandlers.viewErrorHandler());
+
+    AuthSession session = tableSession(response.config(), session(context));
+    ViewMetadata metadata = response.metadata();
+
+    RESTViewOperations ops =
+        new RESTViewOperations(
+            client,
+            paths.view(identifier),
+            session::headers,
+            tableFileIO(context, response.config()),
+            metadata);
+
+    trackFileIO(ops);
+
+    return new BaseView(ops, ViewUtil.fullViewName(name(), identifier));
+  }
+
+  @Override
+  public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) {
+    return new RESTViewBuilder(context, identifier);
+  }
+
+  @Override
+  public boolean dropView(SessionContext context, TableIdentifier identifier) {
+    checkViewIdentifierIsValid(identifier);
+
+    try {
+      client.delete(
+          paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler());
+      return true;
+    } catch (NoSuchViewException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) {
+    checkViewIdentifierIsValid(from);
+    checkViewIdentifierIsValid(to);
+
+    RenameTableRequest request =
+        RenameTableRequest.builder().withSource(from).withDestination(to).build();
+
+    client.post(
+        paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler());
+  }
+
+  private class RESTViewBuilder implements ViewBuilder {
+    private final SessionContext context;
+    private final TableIdentifier identifier;
+    private final ImmutableViewVersion.Builder viewVersionBuilder = ImmutableViewVersion.builder();

Review Comment:
   Looks like this needs to be updated to match the other view builder's structure.



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java:
##########
@@ -374,4 +385,107 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) {
 
     return ops.current();
   }
+
+  public static BaseView baseView(View view) {
+    Preconditions.checkArgument(view instanceof BaseView, "View must be a BaseView");
+    return (BaseView) view;
+  }
+
+  public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) {
+    return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build();
+  }
+
+  public static LoadViewResponse createView(
+      ViewCatalog catalog, Namespace namespace, CreateViewRequest request) {
+    request.validate();
+
+    ViewMetadata viewMetadata = request.metadata();
+    ViewBuilder viewBuilder =
+        catalog
+            .buildView(TableIdentifier.of(namespace, request.name()))
+            .withSchema(viewMetadata.schema())
+            .withProperties(viewMetadata.properties())
+            .withDefaultNamespace(viewMetadata.currentVersion().defaultNamespace())
+            .withDefaultCatalog(viewMetadata.currentVersion().defaultCatalog());
+    viewMetadata.currentVersion().representations().stream()
+        .filter(r -> r instanceof SQLViewRepresentation)
+        .map(r -> (SQLViewRepresentation) r)
+        .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql()));
+    View view = viewBuilder.create();
+
+    return viewResponse(view);
+  }
+
+  private static LoadViewResponse viewResponse(View view) {
+    ViewMetadata metadata = baseView(view).operations().current();
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    View view = catalog.loadView(viewIdentifier);
+    return viewResponse(view);
+  }
+
+  public static LoadViewResponse updateView(
+      ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) {
+    View view = catalog.loadView(ident);
+    ViewMetadata metadata = commit(baseView(view).operations(), request);
+
+    return ImmutableLoadViewResponse.builder()
+        .metadata(metadata)
+        .metadataLocation(metadata.metadataFileLocation())
+        .build();
+  }
+
+  public static void renameView(ViewCatalog catalog, RenameTableRequest request) {
+    catalog.renameView(request.source(), request.destination());
+  }
+
+  public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) {
+    boolean dropped = catalog.dropView(viewIdentifier);
+    if (!dropped) {
+      throw new NoSuchViewException("View does not exist: %s", viewIdentifier);
+    }
+  }
+
+  static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) {
+    AtomicBoolean isRetry = new AtomicBoolean(false);
+    try {
+      Tasks.foreach(ops)
+          .retry(COMMIT_NUM_RETRIES_DEFAULT)
+          .exponentialBackoff(
+              COMMIT_MIN_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_MAX_RETRY_WAIT_MS_DEFAULT,
+              COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT,
+              2.0 /* exponential */)
+          .onlyRetryOn(CommitFailedException.class)
+          .run(
+              taskOps -> {
+                ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current();
+                isRetry.set(true);
+
+                // apply changes
+                ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base);
+                request.updates().forEach(update -> update.applyTo(metadataBuilder));

Review Comment:
   based on https://github.com/apache/iceberg/pull/8147#discussion_r1274225077 we decided that we don't need view requirements (for now), hence this code wasn't validating any requirements. 



-- 
This is an automated message from the 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


Re: [PR] Core: Add View support for REST catalog [iceberg]

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


##########
open-api/rest-catalog-open-api.yaml:
##########
@@ -1630,6 +1990,102 @@ components:
         metadata-log:
           $ref: '#/components/schemas/MetadataLog'
 
+    SQLViewRepresentation:
+      type: object
+      required:
+        - type
+        - sql
+        - dialect
+      properties:
+        type:
+          type: string
+        sql:
+          type: string
+        dialect:
+          type: string
+
+    ViewRepresentation:
+      oneOf:
+        - $ref: '#/components/schemas/SQLViewRepresentation'
+
+    ViewHistoryEntry:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+
+    ViewVersion:
+      type: object
+      required:
+        - version-id
+        - timestamp-ms
+        - schema-id
+        - summary
+        - representations
+        - default-namespace
+      properties:
+        version-id:
+          type: integer
+        timestamp-ms:
+          type: integer
+          format: int64
+        schema-id:
+          type: integer

Review Comment:
   we don't send a -1 for schema-id because we don't provide a way to set the current schema id like we do for table metadata



-- 
This is an automated message from the 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