From 13c92f9b7843a5f0aae08f57261dcf57711c6fa3 Mon Sep 17 00:00:00 2001 From: JUN Date: Sat, 16 Nov 2024 21:55:35 +0800 Subject: [PATCH] [#5438] feat(iceberg): add view event for Iceberg REST server (#5584) ### What changes were proposed in this pull request? Add view event for Iceberg REST server ### Why are the changes needed? Close: #5438 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? unit test --- .../apache/gravitino/iceberg/RESTService.java | 8 + .../IcebergViewEventDispatcher.java | 210 ++++++++++++++++++ .../IcebergViewOperationDispatcher.java | 103 +++++++++ .../IcebergViewOperationExecutor.java | 90 ++++++++ .../service/rest/IcebergViewOperations.java | 121 +++++++--- .../rest/IcebergViewRenameOperations.java | 35 ++- .../api/event/IcebergCreateViewEvent.java | 54 +++++ .../event/IcebergCreateViewFailureEvent.java | 45 ++++ .../api/event/IcebergCreateViewPreEvent.java | 42 ++++ .../api/event/IcebergDropViewEvent.java | 33 +++ .../event/IcebergDropViewFailureEvent.java | 33 +++ .../api/event/IcebergDropViewPreEvent.java | 33 +++ .../api/event/IcebergListViewEvent.java | 38 ++++ .../event/IcebergListViewFailureEvent.java | 32 +++ .../api/event/IcebergListViewPreEvent.java | 32 +++ .../api/event/IcebergLoadViewEvent.java | 44 ++++ .../event/IcebergLoadViewFailureEvent.java | 32 +++ .../api/event/IcebergLoadViewPreEvent.java | 32 +++ .../api/event/IcebergRenameViewEvent.java | 44 ++++ .../event/IcebergRenameViewFailureEvent.java | 43 ++++ .../api/event/IcebergRenameViewPreEvent.java | 42 ++++ .../api/event/IcebergReplaceViewEvent.java | 54 +++++ .../event/IcebergReplaceViewFailureEvent.java | 45 ++++ .../api/event/IcebergReplaceViewPreEvent.java | 42 ++++ .../api/event/IcebergTableFailureEvent.java | 2 +- .../listener/api/event/IcebergViewEvent.java | 30 +++ .../api/event/IcebergViewExistsEvent.java | 41 ++++ .../event/IcebergViewExistsFailureEvent.java | 32 +++ .../api/event/IcebergViewExistsPreEvent.java | 32 +++ .../api/event/IcebergViewFailureEvent.java | 32 +++ .../api/event/IcebergViewPreEvent.java | 31 +++ .../service/rest/IcebergRestTestUtil.java | 9 + .../rest/MockIcebergViewOperations.java | 38 ++++ .../rest/MockIcebergViewRenameOperations.java | 37 +++ .../rest/TestIcebergViewOperations.java | 83 ++++++- 35 files changed, 1613 insertions(+), 41 deletions(-) create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewEventDispatcher.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewOperationDispatcher.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewOperationExecutor.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewFailureEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewPreEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewFailureEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewPreEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewFailureEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewPreEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewFailureEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewPreEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewFailureEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewPreEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewFailureEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewPreEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsFailureEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsPreEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewFailureEvent.java create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewPreEvent.java create mode 100644 iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/MockIcebergViewOperations.java create mode 100644 iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/MockIcebergViewRenameOperations.java diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java index b301204bd20..d8fdc26f32b 100644 --- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java @@ -31,6 +31,9 @@ import org.apache.gravitino.iceberg.service.dispatcher.IcebergTableEventDispatcher; import org.apache.gravitino.iceberg.service.dispatcher.IcebergTableOperationDispatcher; import org.apache.gravitino.iceberg.service.dispatcher.IcebergTableOperationExecutor; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewEventDispatcher; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewOperationDispatcher; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewOperationExecutor; import org.apache.gravitino.iceberg.service.metrics.IcebergMetricsManager; import org.apache.gravitino.iceberg.service.provider.IcebergConfigProvider; import org.apache.gravitino.iceberg.service.provider.IcebergConfigProviderFactory; @@ -90,6 +93,10 @@ private void initServer(IcebergConfig icebergConfig) { new IcebergTableOperationExecutor(icebergCatalogWrapperManager); IcebergTableEventDispatcher icebergTableEventDispatcher = new IcebergTableEventDispatcher(icebergTableOperationExecutor, eventBus, metalakeName); + IcebergViewOperationExecutor icebergViewOperationExecutor = + new IcebergViewOperationExecutor(icebergCatalogWrapperManager); + IcebergViewEventDispatcher icebergViewEventDispatcher = + new IcebergViewEventDispatcher(icebergViewOperationExecutor, eventBus, metalakeName); config.register( new AbstractBinder() { @@ -98,6 +105,7 @@ protected void configure() { bind(icebergCatalogWrapperManager).to(IcebergCatalogWrapperManager.class).ranked(1); bind(icebergMetricsManager).to(IcebergMetricsManager.class).ranked(1); bind(icebergTableEventDispatcher).to(IcebergTableOperationDispatcher.class).ranked(1); + bind(icebergViewEventDispatcher).to(IcebergViewOperationDispatcher.class).ranked(1); } }); diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewEventDispatcher.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewEventDispatcher.java new file mode 100644 index 00000000000..89137ea9129 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewEventDispatcher.java @@ -0,0 +1,210 @@ +/* + * 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.gravitino.iceberg.service.dispatcher; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.gravitino.listener.EventBus; +import org.apache.gravitino.listener.api.event.IcebergCreateViewEvent; +import org.apache.gravitino.listener.api.event.IcebergCreateViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergCreateViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergDropViewEvent; +import org.apache.gravitino.listener.api.event.IcebergDropViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergDropViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergListViewEvent; +import org.apache.gravitino.listener.api.event.IcebergListViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergListViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergLoadViewEvent; +import org.apache.gravitino.listener.api.event.IcebergLoadViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergLoadViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergRenameViewEvent; +import org.apache.gravitino.listener.api.event.IcebergRenameViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergRenameViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergReplaceViewEvent; +import org.apache.gravitino.listener.api.event.IcebergReplaceViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergReplaceViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergRequestContext; +import org.apache.gravitino.listener.api.event.IcebergViewExistsEvent; +import org.apache.gravitino.listener.api.event.IcebergViewExistsFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergViewExistsPreEvent; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.rest.requests.CreateViewRequest; +import org.apache.iceberg.rest.requests.RenameTableRequest; +import org.apache.iceberg.rest.requests.UpdateTableRequest; +import org.apache.iceberg.rest.responses.ListTablesResponse; +import org.apache.iceberg.rest.responses.LoadViewResponse; + +/** + * {@code IcebergViewEventDispatcher} is a decorator for {@link IcebergViewOperationExecutor} that + * not only delegates view operations to the underlying dispatcher but also dispatches corresponding + * events to an {@link EventBus}. + */ +public class IcebergViewEventDispatcher implements IcebergViewOperationDispatcher { + + private IcebergViewOperationDispatcher icebergViewOperationDispatcher; + private EventBus eventBus; + private String metalakeName; + + public IcebergViewEventDispatcher( + IcebergViewOperationDispatcher icebergViewOperationDispatcher, + EventBus eventBus, + String metalakeName) { + this.icebergViewOperationDispatcher = icebergViewOperationDispatcher; + this.eventBus = eventBus; + this.metalakeName = metalakeName; + } + + @Override + public LoadViewResponse createView( + IcebergRequestContext context, Namespace namespace, CreateViewRequest createViewRequest) { + TableIdentifier viewIdentifier = TableIdentifier.of(namespace, createViewRequest.name()); + NameIdentifier nameIdentifier = + IcebergRestUtils.getGravitinoNameIdentifier( + metalakeName, context.catalogName(), viewIdentifier); + eventBus.dispatchEvent( + new IcebergCreateViewPreEvent(context, nameIdentifier, createViewRequest)); + LoadViewResponse loadViewResponse; + try { + loadViewResponse = + icebergViewOperationDispatcher.createView(context, namespace, createViewRequest); + } catch (Exception e) { + eventBus.dispatchEvent( + new IcebergCreateViewFailureEvent(context, nameIdentifier, createViewRequest, e)); + throw e; + } + eventBus.dispatchEvent( + new IcebergCreateViewEvent(context, nameIdentifier, createViewRequest, loadViewResponse)); + return loadViewResponse; + } + + @Override + public LoadViewResponse replaceView( + IcebergRequestContext context, + TableIdentifier viewIdentifier, + UpdateTableRequest replaceViewRequest) { + NameIdentifier gravitinoNameIdentifier = + IcebergRestUtils.getGravitinoNameIdentifier( + metalakeName, context.catalogName(), viewIdentifier); + eventBus.dispatchEvent( + new IcebergReplaceViewPreEvent(context, gravitinoNameIdentifier, replaceViewRequest)); + LoadViewResponse loadViewResponse; + try { + loadViewResponse = + icebergViewOperationDispatcher.replaceView(context, viewIdentifier, replaceViewRequest); + } catch (Exception e) { + eventBus.dispatchEvent( + new IcebergReplaceViewFailureEvent( + context, gravitinoNameIdentifier, replaceViewRequest, e)); + throw e; + } + eventBus.dispatchEvent( + new IcebergReplaceViewEvent( + context, gravitinoNameIdentifier, replaceViewRequest, loadViewResponse)); + return loadViewResponse; + } + + @Override + public void dropView(IcebergRequestContext context, TableIdentifier viewIdentifier) { + NameIdentifier gravitinoNameIdentifier = + IcebergRestUtils.getGravitinoNameIdentifier( + metalakeName, context.catalogName(), viewIdentifier); + eventBus.dispatchEvent(new IcebergDropViewPreEvent(context, gravitinoNameIdentifier)); + try { + icebergViewOperationDispatcher.dropView(context, viewIdentifier); + } catch (Exception e) { + eventBus.dispatchEvent(new IcebergDropViewFailureEvent(context, gravitinoNameIdentifier, e)); + throw e; + } + eventBus.dispatchEvent(new IcebergDropViewEvent(context, gravitinoNameIdentifier)); + } + + @Override + public LoadViewResponse loadView(IcebergRequestContext context, TableIdentifier viewIdentifier) { + NameIdentifier gravitinoNameIdentifier = + IcebergRestUtils.getGravitinoNameIdentifier( + metalakeName, context.catalogName(), viewIdentifier); + eventBus.dispatchEvent(new IcebergLoadViewPreEvent(context, gravitinoNameIdentifier)); + LoadViewResponse loadViewResponse; + try { + loadViewResponse = icebergViewOperationDispatcher.loadView(context, viewIdentifier); + } catch (Exception e) { + eventBus.dispatchEvent(new IcebergLoadViewFailureEvent(context, gravitinoNameIdentifier, e)); + throw e; + } + eventBus.dispatchEvent( + new IcebergLoadViewEvent(context, gravitinoNameIdentifier, loadViewResponse)); + return loadViewResponse; + } + + @Override + public ListTablesResponse listView(IcebergRequestContext context, Namespace namespace) { + NameIdentifier gravitinoNameIdentifier = + IcebergRestUtils.getGravitinoNameIdentifier(metalakeName, context.catalogName(), namespace); + eventBus.dispatchEvent(new IcebergListViewPreEvent(context, gravitinoNameIdentifier)); + ListTablesResponse listViewsResponse; + try { + listViewsResponse = icebergViewOperationDispatcher.listView(context, namespace); + } catch (Exception e) { + eventBus.dispatchEvent(new IcebergListViewFailureEvent(context, gravitinoNameIdentifier, e)); + throw e; + } + eventBus.dispatchEvent(new IcebergListViewEvent(context, gravitinoNameIdentifier)); + return listViewsResponse; + } + + @Override + public boolean viewExists(IcebergRequestContext context, TableIdentifier viewIdentifier) { + NameIdentifier gravitinoNameIdentifier = + IcebergRestUtils.getGravitinoNameIdentifier( + metalakeName, context.catalogName(), viewIdentifier); + eventBus.dispatchEvent(new IcebergViewExistsPreEvent(context, gravitinoNameIdentifier)); + boolean isExists; + try { + isExists = icebergViewOperationDispatcher.viewExists(context, viewIdentifier); + } catch (Exception e) { + eventBus.dispatchEvent( + new IcebergViewExistsFailureEvent(context, gravitinoNameIdentifier, e)); + throw e; + } + eventBus.dispatchEvent(new IcebergViewExistsEvent(context, gravitinoNameIdentifier, isExists)); + return isExists; + } + + @Override + public void renameView(IcebergRequestContext context, RenameTableRequest renameViewRequest) { + TableIdentifier sourceView = renameViewRequest.source(); + NameIdentifier gravitinoNameIdentifier = + IcebergRestUtils.getGravitinoNameIdentifier( + metalakeName, context.catalogName(), sourceView); + eventBus.dispatchEvent( + new IcebergRenameViewPreEvent(context, gravitinoNameIdentifier, renameViewRequest)); + try { + icebergViewOperationDispatcher.renameView(context, renameViewRequest); + } catch (Exception e) { + eventBus.dispatchEvent( + new IcebergRenameViewFailureEvent( + context, gravitinoNameIdentifier, renameViewRequest, e)); + throw e; + } + eventBus.dispatchEvent( + new IcebergRenameViewEvent(context, gravitinoNameIdentifier, renameViewRequest)); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewOperationDispatcher.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewOperationDispatcher.java new file mode 100644 index 00000000000..2f084a744ce --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewOperationDispatcher.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.gravitino.iceberg.service.dispatcher; + +import org.apache.gravitino.listener.api.event.IcebergRequestContext; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.rest.requests.CreateViewRequest; +import org.apache.iceberg.rest.requests.RenameTableRequest; +import org.apache.iceberg.rest.requests.UpdateTableRequest; +import org.apache.iceberg.rest.responses.ListTablesResponse; +import org.apache.iceberg.rest.responses.LoadViewResponse; + +/** + * The {@code IcebergViewOperationDispatcher} interface defines the public API for managing Iceberg + * views. + */ +public interface IcebergViewOperationDispatcher { + + /** + * Creates a new Iceberg view. + * + * @param context Iceberg REST request context information. + * @param namespace The namespace within which the view should be created. + * @param createViewRequest The request object containing the details for creating the view. + * @return A {@link LoadViewResponse} object containing the result of the operation. + */ + LoadViewResponse createView( + IcebergRequestContext context, Namespace namespace, CreateViewRequest createViewRequest); + + /** + * Updates an Iceberg view. + * + * @param context Iceberg REST request context information. + * @param viewIdentifier The Iceberg view identifier. + * @param replaceViewRequest The request object containing the details for updating the view. + * @return A {@link LoadViewResponse} object containing the result of the operation. + */ + LoadViewResponse replaceView( + IcebergRequestContext context, + TableIdentifier viewIdentifier, + UpdateTableRequest replaceViewRequest); + + /** + * Drops an Iceberg view. + * + * @param context Iceberg REST request context information. + * @param viewIdentifier The Iceberg view identifier. + */ + void dropView(IcebergRequestContext context, TableIdentifier viewIdentifier); + + /** + * Loads an Iceberg view. + * + * @param context Iceberg REST request context information. + * @param viewIdentifier The Iceberg view identifier. + * @return A {@link LoadViewResponse} object containing the result of the operation. + */ + LoadViewResponse loadView(IcebergRequestContext context, TableIdentifier viewIdentifier); + + /** + * Lists Iceberg views. + * + * @param context Iceberg REST request context information. + * @param namespace The Iceberg namespace. + * @return A {@link ListTablesResponse} object containing the list of view identifiers. + */ + ListTablesResponse listView(IcebergRequestContext context, Namespace namespace); + + /** + * Check whether an Iceberg view exists. + * + * @param context Iceberg REST request context information. + * @param viewIdentifier The Iceberg view identifier. + * @return Whether view exists. + */ + boolean viewExists(IcebergRequestContext context, TableIdentifier viewIdentifier); + + /** + * Rename an Iceberg view. + * + * @param context Iceberg REST request context information. + * @param renameViewRequest Rename view request information. + */ + void renameView(IcebergRequestContext context, RenameTableRequest renameViewRequest); +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewOperationExecutor.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewOperationExecutor.java new file mode 100644 index 00000000000..308c20c437c --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/dispatcher/IcebergViewOperationExecutor.java @@ -0,0 +1,90 @@ +/* + * 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.gravitino.iceberg.service.dispatcher; + +import org.apache.gravitino.iceberg.service.IcebergCatalogWrapperManager; +import org.apache.gravitino.listener.api.event.IcebergRequestContext; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.rest.requests.CreateViewRequest; +import org.apache.iceberg.rest.requests.RenameTableRequest; +import org.apache.iceberg.rest.requests.UpdateTableRequest; +import org.apache.iceberg.rest.responses.ListTablesResponse; +import org.apache.iceberg.rest.responses.LoadViewResponse; + +public class IcebergViewOperationExecutor implements IcebergViewOperationDispatcher { + + private IcebergCatalogWrapperManager icebergCatalogWrapperManager; + + public IcebergViewOperationExecutor(IcebergCatalogWrapperManager icebergCatalogWrapperManager) { + this.icebergCatalogWrapperManager = icebergCatalogWrapperManager; + } + + @Override + public LoadViewResponse createView( + IcebergRequestContext context, Namespace namespace, CreateViewRequest createViewRequest) { + return icebergCatalogWrapperManager + .getCatalogWrapper(context.catalogName()) + .createView(namespace, createViewRequest); + } + + @Override + public LoadViewResponse replaceView( + IcebergRequestContext context, + TableIdentifier viewIdentifier, + UpdateTableRequest replaceViewRequest) { + return icebergCatalogWrapperManager + .getCatalogWrapper(context.catalogName()) + .updateView(viewIdentifier, replaceViewRequest); + } + + @Override + public void dropView(IcebergRequestContext context, TableIdentifier viewIdentifier) { + icebergCatalogWrapperManager.getCatalogWrapper(context.catalogName()).dropView(viewIdentifier); + } + + @Override + public LoadViewResponse loadView(IcebergRequestContext context, TableIdentifier viewIdentifier) { + return icebergCatalogWrapperManager + .getCatalogWrapper(context.catalogName()) + .loadView(viewIdentifier); + } + + @Override + public ListTablesResponse listView(IcebergRequestContext context, Namespace namespace) { + return icebergCatalogWrapperManager + .getCatalogWrapper(context.catalogName()) + .listView(namespace); + } + + @Override + public boolean viewExists(IcebergRequestContext context, TableIdentifier viewIdentifier) { + return icebergCatalogWrapperManager + .getCatalogWrapper(context.catalogName()) + .existView(viewIdentifier); + } + + @Override + public void renameView(IcebergRequestContext context, RenameTableRequest renameViewRequest) { + icebergCatalogWrapperManager + .getCatalogWrapper(context.catalogName()) + .renameView(renameViewRequest); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergViewOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergViewOperations.java index 3e46257e22b..26c466504a9 100644 --- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergViewOperations.java +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergViewOperations.java @@ -20,6 +20,9 @@ import com.codahale.metrics.annotation.ResponseMetered; import com.codahale.metrics.annotation.Timed; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import javax.inject.Inject; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -33,30 +36,37 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import org.apache.gravitino.iceberg.service.IcebergCatalogWrapperManager; +import org.apache.gravitino.iceberg.service.IcebergObjectMapper; import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewOperationDispatcher; +import org.apache.gravitino.listener.api.event.IcebergRequestContext; import org.apache.gravitino.metrics.MetricNames; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.rest.RESTUtil; import org.apache.iceberg.rest.requests.CreateViewRequest; import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.responses.ListTablesResponse; import org.apache.iceberg.rest.responses.LoadViewResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Path("/v1/{prefix:([^/]*/)?}namespaces/{namespace}/views") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public class IcebergViewOperations { - private IcebergCatalogWrapperManager icebergCatalogWrapperManager; + private static final Logger LOG = LoggerFactory.getLogger(IcebergViewOperations.class); - @SuppressWarnings("UnusedVariable") - @Context - private HttpServletRequest httpRequest; + private ObjectMapper icebergObjectMapper; + private IcebergViewOperationDispatcher viewOperationDispatcher; + + @Context private HttpServletRequest httpRequest; @Inject - public IcebergViewOperations(IcebergCatalogWrapperManager icebergCatalogWrapperManager) { - this.icebergCatalogWrapperManager = icebergCatalogWrapperManager; + public IcebergViewOperations(IcebergViewOperationDispatcher viewOperationDispatcher) { + this.viewOperationDispatcher = viewOperationDispatcher; + this.icebergObjectMapper = IcebergObjectMapper.getInstance(); } @GET @@ -65,9 +75,12 @@ public IcebergViewOperations(IcebergCatalogWrapperManager icebergCatalogWrapperM @ResponseMetered(name = "list-view", absolute = true) public Response listView( @PathParam("prefix") String prefix, @PathParam("namespace") String namespace) { - ListTablesResponse response = - icebergCatalogWrapperManager.getOps(prefix).listView(RESTUtil.decodeNamespace(namespace)); - return IcebergRestUtils.ok(response); + String catalogName = IcebergRestUtils.getCatalogName(prefix); + Namespace icebergNS = RESTUtil.decodeNamespace(namespace); + LOG.info("List Iceberg views, catalog: {}, namespace: {}", catalogName, icebergNS); + IcebergRequestContext context = new IcebergRequestContext(httpServletRequest(), catalogName); + ListTablesResponse listTablesResponse = viewOperationDispatcher.listView(context, icebergNS); + return IcebergRestUtils.ok(listTablesResponse); } @POST @@ -77,12 +90,19 @@ public Response listView( public Response createView( @PathParam("prefix") String prefix, @PathParam("namespace") String namespace, - CreateViewRequest request) { - LoadViewResponse response = - icebergCatalogWrapperManager - .getOps(prefix) - .createView(RESTUtil.decodeNamespace(namespace), request); - return IcebergRestUtils.ok(response); + CreateViewRequest createViewRequest) { + String catalogName = IcebergRestUtils.getCatalogName(prefix); + Namespace icebergNS = RESTUtil.decodeNamespace(namespace); + LOG.info( + "Create Iceberg view, catalog: {}, namespace: {}, createViewRequest: {}", + catalogName, + icebergNS, + createViewRequest); + IcebergRequestContext context = new IcebergRequestContext(httpServletRequest(), catalogName); + LoadViewResponse loadViewResponse = + viewOperationDispatcher.createView(context, icebergNS, createViewRequest); + + return IcebergRestUtils.ok(loadViewResponse); } @GET @@ -94,10 +114,15 @@ public Response loadView( @PathParam("prefix") String prefix, @PathParam("namespace") String namespace, @PathParam("view") String view) { - TableIdentifier viewIdentifier = TableIdentifier.of(RESTUtil.decodeNamespace(namespace), view); - LoadViewResponse response = - icebergCatalogWrapperManager.getOps(prefix).loadView(viewIdentifier); - return IcebergRestUtils.ok(response); + String catalogName = IcebergRestUtils.getCatalogName(prefix); + Namespace icebergNS = RESTUtil.decodeNamespace(namespace); + LOG.info( + "Load Iceberg view, catalog: {}, namespace: {}, view: {}", catalogName, icebergNS, view); + + TableIdentifier viewIdentifier = TableIdentifier.of(icebergNS, view); + IcebergRequestContext context = new IcebergRequestContext(httpServletRequest(), catalogName); + LoadViewResponse loadViewResponse = viewOperationDispatcher.loadView(context, viewIdentifier); + return IcebergRestUtils.ok(loadViewResponse); } @POST @@ -109,11 +134,20 @@ public Response replaceView( @PathParam("prefix") String prefix, @PathParam("namespace") String namespace, @PathParam("view") String view, - UpdateTableRequest request) { - TableIdentifier viewIdentifier = TableIdentifier.of(RESTUtil.decodeNamespace(namespace), view); - LoadViewResponse response = - icebergCatalogWrapperManager.getOps(prefix).updateView(viewIdentifier, request); - return IcebergRestUtils.ok(response); + UpdateTableRequest replaceViewRequest) { + String catalogName = IcebergRestUtils.getCatalogName(prefix); + Namespace icebergNS = RESTUtil.decodeNamespace(namespace); + LOG.info( + "Replace Iceberg view, catalog: {}, namespace: {}, table: {}, replaceViewRequest: {}", + catalogName, + icebergNS, + view, + SerializeReplaceViewRequest(replaceViewRequest)); + IcebergRequestContext context = new IcebergRequestContext(httpServletRequest(), catalogName); + TableIdentifier viewIdentifier = TableIdentifier.of(icebergNS, view); + LoadViewResponse loadViewResponse = + viewOperationDispatcher.replaceView(context, viewIdentifier, replaceViewRequest); + return IcebergRestUtils.ok(loadViewResponse); } @DELETE @@ -125,8 +159,13 @@ public Response dropView( @PathParam("prefix") String prefix, @PathParam("namespace") String namespace, @PathParam("view") String view) { - TableIdentifier viewIdentifier = TableIdentifier.of(RESTUtil.decodeNamespace(namespace), view); - icebergCatalogWrapperManager.getOps(prefix).dropView(viewIdentifier); + String catalogName = IcebergRestUtils.getCatalogName(prefix); + Namespace icebergNS = RESTUtil.decodeNamespace(namespace); + LOG.info( + "Drop Iceberg view, catalog: {}, namespace: {}, table: {}", catalogName, icebergNS, view); + TableIdentifier viewIdentifier = TableIdentifier.of(namespace, view); + IcebergRequestContext context = new IcebergRequestContext(httpServletRequest(), catalogName); + viewOperationDispatcher.dropView(context, viewIdentifier); return IcebergRestUtils.noContent(); } @@ -139,11 +178,35 @@ public Response viewExists( @PathParam("prefix") String prefix, @PathParam("namespace") String namespace, @PathParam("view") String view) { - TableIdentifier tableIdentifier = TableIdentifier.of(RESTUtil.decodeNamespace(namespace), view); - if (icebergCatalogWrapperManager.getOps(prefix).existView(tableIdentifier)) { + String catalogName = IcebergRestUtils.getCatalogName(prefix); + Namespace icebergNS = RESTUtil.decodeNamespace(namespace); + LOG.info( + "Check Iceberg view exists, catalog: {}, namespace: {}, table: {}", + catalogName, + icebergNS, + view); + IcebergRequestContext context = new IcebergRequestContext(httpServletRequest(), catalogName); + TableIdentifier viewIdentifier = TableIdentifier.of(icebergNS, view); + boolean exists = viewOperationDispatcher.viewExists(context, viewIdentifier); + if (exists) { return IcebergRestUtils.noContent(); } else { return IcebergRestUtils.notExists(); } } + + // HTTP request is null in Jersey test, override with a mock request when testing. + @VisibleForTesting + HttpServletRequest httpServletRequest() { + return httpRequest; + } + + private String SerializeReplaceViewRequest(UpdateTableRequest replaceViewRequest) { + try { + return icebergObjectMapper.writeValueAsString(replaceViewRequest); + } catch (JsonProcessingException e) { + LOG.warn("Serialize update view request failed", e); + return replaceViewRequest.toString(); + } + } } diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergViewRenameOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergViewRenameOperations.java index 128689d33be..4c6f706086b 100644 --- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergViewRenameOperations.java +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergViewRenameOperations.java @@ -20,6 +20,7 @@ import com.codahale.metrics.annotation.ResponseMetered; import com.codahale.metrics.annotation.Timed; +import com.google.common.annotations.VisibleForTesting; import javax.inject.Inject; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -30,33 +31,49 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import org.apache.gravitino.iceberg.service.IcebergCatalogWrapperManager; import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewOperationDispatcher; +import org.apache.gravitino.listener.api.event.IcebergRequestContext; import org.apache.gravitino.metrics.MetricNames; import org.apache.iceberg.rest.requests.RenameTableRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @Path("/v1/{prefix:([^/]*/)?}views/rename") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public class IcebergViewRenameOperations { + private static final Logger LOG = LoggerFactory.getLogger(IcebergViewRenameOperations.class); - @SuppressWarnings("UnusedVariable") - @Context - private HttpServletRequest httpRequest; + @Context private HttpServletRequest httpRequest; - private IcebergCatalogWrapperManager icebergCatalogWrapperManager; + private IcebergViewOperationDispatcher viewOperationDispatcher; @Inject - public IcebergViewRenameOperations(IcebergCatalogWrapperManager icebergCatalogWrapperManager) { - this.icebergCatalogWrapperManager = icebergCatalogWrapperManager; + public IcebergViewRenameOperations(IcebergViewOperationDispatcher viewOperationDispatcher) { + this.viewOperationDispatcher = viewOperationDispatcher; } @POST @Produces(MediaType.APPLICATION_JSON) @Timed(name = "rename-view." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) @ResponseMetered(name = "rename-view", absolute = true) - public Response renameView(@PathParam("prefix") String prefix, RenameTableRequest request) { - icebergCatalogWrapperManager.getOps(prefix).renameView(request); + public Response renameView( + @PathParam("prefix") String prefix, RenameTableRequest renameViewRequest) { + String catalogName = IcebergRestUtils.getCatalogName(prefix); + LOG.info( + "Rename Iceberg view, catalog: {}, source: {}, destination: {}.", + catalogName, + renameViewRequest.source(), + renameViewRequest.destination()); + IcebergRequestContext context = new IcebergRequestContext(httpServletRequest(), catalogName); + viewOperationDispatcher.renameView(context, renameViewRequest); return IcebergRestUtils.noContent(); } + + // HTTP request is null in Jersey test, override with a mock request when testing. + @VisibleForTesting + HttpServletRequest httpServletRequest() { + return httpRequest; + } } diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewEvent.java new file mode 100644 index 00000000000..ff9f306c209 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewEvent.java @@ -0,0 +1,54 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.iceberg.rest.requests.CreateViewRequest; +import org.apache.iceberg.rest.responses.LoadViewResponse; + +/** Represent an event after creating Iceberg view successfully. */ +@DeveloperApi +public class IcebergCreateViewEvent extends IcebergViewEvent { + + private final CreateViewRequest createViewRequest; + private final LoadViewResponse loadViewResponse; + + public IcebergCreateViewEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + CreateViewRequest createViewRequest, + LoadViewResponse loadViewResponse) { + super(icebergRequestContext, viewIdentifier); + this.createViewRequest = + IcebergRestUtils.cloneIcebergRESTObject(createViewRequest, CreateViewRequest.class); + this.loadViewResponse = + IcebergRestUtils.cloneIcebergRESTObject(loadViewResponse, LoadViewResponse.class); + } + + public CreateViewRequest createViewRequest() { + return createViewRequest; + } + + public LoadViewResponse loadViewResponse() { + return loadViewResponse; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewFailureEvent.java new file mode 100644 index 00000000000..4da73ed18ba --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewFailureEvent.java @@ -0,0 +1,45 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.iceberg.rest.requests.CreateViewRequest; + +/** Represent a failure event when creating Iceberg view failed. */ +@DeveloperApi +public class IcebergCreateViewFailureEvent extends IcebergViewFailureEvent { + private final CreateViewRequest createViewRequest; + + public IcebergCreateViewFailureEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + CreateViewRequest createViewRequest, + Exception e) { + super(icebergRequestContext, viewIdentifier, e); + this.createViewRequest = + IcebergRestUtils.cloneIcebergRESTObject(createViewRequest, CreateViewRequest.class); + } + + public CreateViewRequest createViewRequest() { + return createViewRequest; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewPreEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewPreEvent.java new file mode 100644 index 00000000000..00eea5db73d --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergCreateViewPreEvent.java @@ -0,0 +1,42 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.iceberg.rest.requests.CreateViewRequest; + +/** Represent a pre event before creating Iceberg view. */ +@DeveloperApi +public class IcebergCreateViewPreEvent extends IcebergViewPreEvent { + private final CreateViewRequest createViewRequest; + + public IcebergCreateViewPreEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + CreateViewRequest createViewRequest) { + super(icebergRequestContext, viewIdentifier); + this.createViewRequest = createViewRequest; + } + + public CreateViewRequest createViewRequest() { + return createViewRequest; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewEvent.java new file mode 100644 index 00000000000..7a17ca1f472 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewEvent.java @@ -0,0 +1,33 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent an event after dropping Iceberg view successfully. */ +@DeveloperApi +public class IcebergDropViewEvent extends IcebergViewEvent { + + public IcebergDropViewEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier) { + super(icebergRequestContext, viewIdentifier); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewFailureEvent.java new file mode 100644 index 00000000000..99502160b44 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewFailureEvent.java @@ -0,0 +1,33 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a failure event when dropping Iceberg view failed. */ +@DeveloperApi +public class IcebergDropViewFailureEvent extends IcebergViewFailureEvent { + + public IcebergDropViewFailureEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier, Exception e) { + super(icebergRequestContext, viewIdentifier, e); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewPreEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewPreEvent.java new file mode 100644 index 00000000000..163a51e9b60 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergDropViewPreEvent.java @@ -0,0 +1,33 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a pre event before dropping Iceberg view. */ +@DeveloperApi +public class IcebergDropViewPreEvent extends IcebergViewPreEvent { + + public IcebergDropViewPreEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier) { + super(icebergRequestContext, viewIdentifier); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewEvent.java new file mode 100644 index 00000000000..6bc7d83ac64 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewEvent.java @@ -0,0 +1,38 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** + * Represent an event after listing Iceberg view successfully. + * + *

To optimize memory usage and avoid the potential overhead associated with storing a large + * number of views directly within the ListViewEvent, the actual views listed are not maintained in + * this event. + */ +@DeveloperApi +public class IcebergListViewEvent extends IcebergViewEvent { + public IcebergListViewEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier) { + super(icebergRequestContext, viewIdentifier); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewFailureEvent.java new file mode 100644 index 00000000000..400e6b43039 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewFailureEvent.java @@ -0,0 +1,32 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a failure event when listing Iceberg view failed. */ +@DeveloperApi +public class IcebergListViewFailureEvent extends IcebergViewFailureEvent { + public IcebergListViewFailureEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier, Exception e) { + super(icebergRequestContext, viewIdentifier, e); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewPreEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewPreEvent.java new file mode 100644 index 00000000000..f0bebca38b0 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergListViewPreEvent.java @@ -0,0 +1,32 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a pre event before listing Iceberg view. */ +@DeveloperApi +public class IcebergListViewPreEvent extends IcebergViewPreEvent { + public IcebergListViewPreEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier) { + super(icebergRequestContext, viewIdentifier); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewEvent.java new file mode 100644 index 00000000000..5f22fe1b62e --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewEvent.java @@ -0,0 +1,44 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.iceberg.rest.responses.LoadViewResponse; + +/** Represent an event after loading Iceberg view successfully. */ +@DeveloperApi +public class IcebergLoadViewEvent extends IcebergViewEvent { + private final LoadViewResponse loadViewResponse; + + public IcebergLoadViewEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + LoadViewResponse loadViewResponse) { + super(icebergRequestContext, viewIdentifier); + this.loadViewResponse = + IcebergRestUtils.cloneIcebergRESTObject(loadViewResponse, LoadViewResponse.class); + } + + public LoadViewResponse loadViewResponse() { + return loadViewResponse; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewFailureEvent.java new file mode 100644 index 00000000000..493c9a3c1e7 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewFailureEvent.java @@ -0,0 +1,32 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a failure event when loading Iceberg view failed. */ +@DeveloperApi +public class IcebergLoadViewFailureEvent extends IcebergViewFailureEvent { + public IcebergLoadViewFailureEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier, Exception e) { + super(icebergRequestContext, viewIdentifier, e); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewPreEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewPreEvent.java new file mode 100644 index 00000000000..5970edc69d6 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergLoadViewPreEvent.java @@ -0,0 +1,32 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a pre event before loading Iceberg view. */ +@DeveloperApi +public class IcebergLoadViewPreEvent extends IcebergViewPreEvent { + public IcebergLoadViewPreEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier) { + super(icebergRequestContext, viewIdentifier); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewEvent.java new file mode 100644 index 00000000000..985a35592bb --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewEvent.java @@ -0,0 +1,44 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.iceberg.rest.requests.RenameTableRequest; + +/** Represent an event after rename Iceberg view successfully. */ +@DeveloperApi +public class IcebergRenameViewEvent extends IcebergViewEvent { + private final RenameTableRequest renameViewRequest; + + public IcebergRenameViewEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + RenameTableRequest renameViewRequest) { + super(icebergRequestContext, viewIdentifier); + this.renameViewRequest = + IcebergRestUtils.cloneIcebergRESTObject(renameViewRequest, RenameTableRequest.class); + } + + public RenameTableRequest renameViewRequest() { + return renameViewRequest; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewFailureEvent.java new file mode 100644 index 00000000000..f5b943fabc9 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewFailureEvent.java @@ -0,0 +1,43 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.iceberg.rest.requests.RenameTableRequest; + +/** Represent an event when rename Iceberg view failed. */ +@DeveloperApi +public class IcebergRenameViewFailureEvent extends IcebergViewFailureEvent { + private final RenameTableRequest renameViewRequest; + + public IcebergRenameViewFailureEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + RenameTableRequest renameViewRequest, + Exception e) { + super(icebergRequestContext, viewIdentifier, e); + this.renameViewRequest = renameViewRequest; + } + + public RenameTableRequest renameViewRequest() { + return renameViewRequest; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewPreEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewPreEvent.java new file mode 100644 index 00000000000..7802b853429 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergRenameViewPreEvent.java @@ -0,0 +1,42 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.iceberg.rest.requests.RenameTableRequest; + +/** Represent an pre event before rename an Iceberg view. */ +@DeveloperApi +public class IcebergRenameViewPreEvent extends IcebergViewPreEvent { + private final RenameTableRequest renameViewRequest; + + public IcebergRenameViewPreEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + RenameTableRequest renameViewRequest) { + super(icebergRequestContext, viewIdentifier); + this.renameViewRequest = renameViewRequest; + } + + public RenameTableRequest renameViewRequest() { + return renameViewRequest; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewEvent.java new file mode 100644 index 00000000000..9f3f1677eee --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewEvent.java @@ -0,0 +1,54 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.iceberg.rest.requests.UpdateTableRequest; +import org.apache.iceberg.rest.responses.LoadViewResponse; + +/** Represent an event after updating Iceberg view successfully. */ +@DeveloperApi +public class IcebergReplaceViewEvent extends IcebergViewEvent { + + private final UpdateTableRequest replaceViewRequest; + private final LoadViewResponse loadViewResponse; + + public IcebergReplaceViewEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + UpdateTableRequest replaceViewRequest, + LoadViewResponse loadViewResponse) { + super(icebergRequestContext, viewIdentifier); + this.replaceViewRequest = + IcebergRestUtils.cloneIcebergRESTObject(replaceViewRequest, UpdateTableRequest.class); + this.loadViewResponse = + IcebergRestUtils.cloneIcebergRESTObject(loadViewResponse, LoadViewResponse.class); + } + + public UpdateTableRequest renameViewRequest() { + return replaceViewRequest; + } + + public LoadViewResponse loadViewResponse() { + return loadViewResponse; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewFailureEvent.java new file mode 100644 index 00000000000..b23e028aadd --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewFailureEvent.java @@ -0,0 +1,45 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.gravitino.iceberg.service.IcebergRestUtils; +import org.apache.iceberg.rest.requests.UpdateTableRequest; + +/** Represent a failure event when updating Iceberg view failed. */ +@DeveloperApi +public class IcebergReplaceViewFailureEvent extends IcebergViewFailureEvent { + private final UpdateTableRequest replaceViewRequest; + + public IcebergReplaceViewFailureEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + UpdateTableRequest replaceViewRequest, + Exception e) { + super(icebergRequestContext, viewIdentifier, e); + this.replaceViewRequest = + IcebergRestUtils.cloneIcebergRESTObject(replaceViewRequest, UpdateTableRequest.class); + } + + public UpdateTableRequest replaceViewRequest() { + return replaceViewRequest; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewPreEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewPreEvent.java new file mode 100644 index 00000000000..aa15593aba6 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergReplaceViewPreEvent.java @@ -0,0 +1,42 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; +import org.apache.iceberg.rest.requests.UpdateTableRequest; + +/** Represent a pre event before updating Iceberg view. */ +@DeveloperApi +public class IcebergReplaceViewPreEvent extends IcebergViewPreEvent { + private final UpdateTableRequest replaceViewRequest; + + public IcebergReplaceViewPreEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + UpdateTableRequest replaceViewRequest) { + super(icebergRequestContext, viewIdentifier); + this.replaceViewRequest = replaceViewRequest; + } + + public UpdateTableRequest replaceViewRequest() { + return replaceViewRequest; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergTableFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergTableFailureEvent.java index 4e2187f1a99..28c3fe08806 100644 --- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergTableFailureEvent.java +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergTableFailureEvent.java @@ -24,7 +24,7 @@ /** Represent a failure event when do Iceberg table operation failed. */ @DeveloperApi -public class IcebergTableFailureEvent extends IcebergFailureEvent { +public abstract class IcebergTableFailureEvent extends IcebergFailureEvent { protected IcebergTableFailureEvent( IcebergRequestContext icebergRequestContext, NameIdentifier nameIdentifier, Exception e) { super(icebergRequestContext, nameIdentifier, e); diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewEvent.java new file mode 100644 index 00000000000..2fd3c735d3a --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewEvent.java @@ -0,0 +1,30 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; + +/** Represents an abstract view post event in Gravitino Iceberg REST server. */ +public abstract class IcebergViewEvent extends IcebergEvent { + protected IcebergViewEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier) { + super(icebergRequestContext, viewIdentifier); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsEvent.java new file mode 100644 index 00000000000..ece8523208c --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsEvent.java @@ -0,0 +1,41 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent an event after check Iceberg view exists successfully. */ +@DeveloperApi +public class IcebergViewExistsEvent extends IcebergViewEvent { + private final boolean isExists; + + public IcebergViewExistsEvent( + IcebergRequestContext icebergRequestContext, + NameIdentifier viewIdentifier, + boolean isExists) { + super(icebergRequestContext, viewIdentifier); + this.isExists = isExists; + } + + public boolean isExists() { + return isExists; + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsFailureEvent.java new file mode 100644 index 00000000000..93aa839d400 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsFailureEvent.java @@ -0,0 +1,32 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a failure event when check Iceberg view exist failed. */ +@DeveloperApi +public class IcebergViewExistsFailureEvent extends IcebergViewFailureEvent { + public IcebergViewExistsFailureEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier, Exception e) { + super(icebergRequestContext, viewIdentifier, e); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsPreEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsPreEvent.java new file mode 100644 index 00000000000..938a7587ecb --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewExistsPreEvent.java @@ -0,0 +1,32 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a pre event before checking Iceberg view exists. */ +@DeveloperApi +public class IcebergViewExistsPreEvent extends IcebergViewPreEvent { + public IcebergViewExistsPreEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier) { + super(icebergRequestContext, viewIdentifier); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewFailureEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewFailureEvent.java new file mode 100644 index 00000000000..4122bc32334 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewFailureEvent.java @@ -0,0 +1,32 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represent a failure event when do Iceberg view operation failed. */ +@DeveloperApi +public abstract class IcebergViewFailureEvent extends IcebergFailureEvent { + protected IcebergViewFailureEvent( + IcebergRequestContext icebergRequestContext, NameIdentifier viewIdentifier, Exception e) { + super(icebergRequestContext, viewIdentifier, e); + } +} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewPreEvent.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewPreEvent.java new file mode 100644 index 00000000000..2deaf49c370 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/listener/api/event/IcebergViewPreEvent.java @@ -0,0 +1,31 @@ +/* + * 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.gravitino.listener.api.event; + +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.annotation.DeveloperApi; + +/** Represents an abstract view pre event in Gravitino Iceberg REST server. */ +@DeveloperApi +public abstract class IcebergViewPreEvent extends IcebergPreEvent { + protected IcebergViewPreEvent(IcebergRequestContext context, NameIdentifier viewIdentifier) { + super(context, viewIdentifier); + } +} diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java index a2ee3d888ab..56b7fa2b023 100644 --- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java +++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java @@ -39,6 +39,9 @@ import org.apache.gravitino.iceberg.service.dispatcher.IcebergTableEventDispatcher; import org.apache.gravitino.iceberg.service.dispatcher.IcebergTableOperationDispatcher; import org.apache.gravitino.iceberg.service.dispatcher.IcebergTableOperationExecutor; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewEventDispatcher; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewOperationDispatcher; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewOperationExecutor; import org.apache.gravitino.iceberg.service.extension.DummyCredentialProvider; import org.apache.gravitino.iceberg.service.metrics.IcebergMetricsManager; import org.apache.gravitino.iceberg.service.provider.IcebergConfigProvider; @@ -113,6 +116,11 @@ public static ResourceConfig getIcebergResourceConfig( IcebergTableEventDispatcher icebergTableEventDispatcher = new IcebergTableEventDispatcher( icebergTableOperationExecutor, eventBus, configProvider.getMetalakeName()); + IcebergViewOperationExecutor icebergViewOperationExecutor = + new IcebergViewOperationExecutor(icebergCatalogWrapperManager); + IcebergViewEventDispatcher icebergViewEventDispatcher = + new IcebergViewEventDispatcher( + icebergViewOperationExecutor, eventBus, configProvider.getMetalakeName()); IcebergMetricsManager icebergMetricsManager = new IcebergMetricsManager(new IcebergConfig()); resourceConfig.register( @@ -122,6 +130,7 @@ protected void configure() { bind(icebergCatalogWrapperManager).to(IcebergCatalogWrapperManager.class).ranked(2); bind(icebergMetricsManager).to(IcebergMetricsManager.class).ranked(2); bind(icebergTableEventDispatcher).to(IcebergTableOperationDispatcher.class).ranked(2); + bind(icebergViewEventDispatcher).to(IcebergViewOperationDispatcher.class).ranked(2); } }); } diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/MockIcebergViewOperations.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/MockIcebergViewOperations.java new file mode 100644 index 00000000000..ed229a0424f --- /dev/null +++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/MockIcebergViewOperations.java @@ -0,0 +1,38 @@ +/* + * 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.gravitino.iceberg.service.rest; + +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewOperationDispatcher; + +public class MockIcebergViewOperations extends IcebergViewOperations { + + @Inject + public MockIcebergViewOperations(IcebergViewOperationDispatcher viewOperationDispatcher) { + super(viewOperationDispatcher); + } + + // HTTP request is null in Jersey test, create a mock request + @Override + HttpServletRequest httpServletRequest() { + return IcebergRestTestUtil.createMockHttpRequest(); + } +} diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/MockIcebergViewRenameOperations.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/MockIcebergViewRenameOperations.java new file mode 100644 index 00000000000..694db5e2012 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/MockIcebergViewRenameOperations.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.gravitino.iceberg.service.rest; + +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import org.apache.gravitino.iceberg.service.dispatcher.IcebergViewOperationDispatcher; + +public class MockIcebergViewRenameOperations extends IcebergViewRenameOperations { + @Inject + public MockIcebergViewRenameOperations(IcebergViewOperationDispatcher viewOperationDispatcher) { + super(viewOperationDispatcher); + } + + // HTTP request is null in Jersey test, create a mock request + @Override + HttpServletRequest httpServletRequest() { + return IcebergRestTestUtil.createMockHttpRequest(); + } +} diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergViewOperations.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergViewOperations.java index 9ec2dc66f46..df896b1529b 100644 --- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergViewOperations.java +++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergViewOperations.java @@ -20,6 +20,7 @@ package org.apache.gravitino.iceberg.service.rest; import com.google.common.collect.ImmutableSet; +import java.util.Arrays; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -27,6 +28,27 @@ import javax.ws.rs.core.Application; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import org.apache.gravitino.listener.api.event.Event; +import org.apache.gravitino.listener.api.event.IcebergCreateViewEvent; +import org.apache.gravitino.listener.api.event.IcebergCreateViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergCreateViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergDropViewEvent; +import org.apache.gravitino.listener.api.event.IcebergDropViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergDropViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergListViewEvent; +import org.apache.gravitino.listener.api.event.IcebergListViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergListViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergLoadViewEvent; +import org.apache.gravitino.listener.api.event.IcebergLoadViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergLoadViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergRenameViewEvent; +import org.apache.gravitino.listener.api.event.IcebergRenameViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergRenameViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergReplaceViewEvent; +import org.apache.gravitino.listener.api.event.IcebergReplaceViewFailureEvent; +import org.apache.gravitino.listener.api.event.IcebergReplaceViewPreEvent; +import org.apache.gravitino.listener.api.event.IcebergViewExistsEvent; +import org.apache.gravitino.listener.api.event.IcebergViewExistsPreEvent; import org.apache.iceberg.Schema; import org.apache.iceberg.UpdateRequirements; import org.apache.iceberg.catalog.Namespace; @@ -56,13 +78,17 @@ public class TestIcebergViewOperations extends TestIcebergNamespaceOperations { private static final String VIEW_QUERY = "select 1"; + private DummyEventListener dummyEventListener; + @Override protected Application configure() { + this.dummyEventListener = new DummyEventListener(); ResourceConfig resourceConfig = - IcebergRestTestUtil.getIcebergResourceConfig(IcebergViewOperations.class); + IcebergRestTestUtil.getIcebergResourceConfig( + MockIcebergViewOperations.class, true, Arrays.asList(dummyEventListener)); // create namespace before each view test resourceConfig.register(IcebergNamespaceOperations.class); - resourceConfig.register(IcebergViewRenameOperations.class); + resourceConfig.register(MockIcebergViewRenameOperations.class); return resourceConfig; } @@ -72,20 +98,31 @@ protected Application configure() { void testListViews(String prefix) { setUrlPathWithPrefix(prefix); verifyListViewFail(404); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergListViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergListViewFailureEvent); verifyCreateNamespaceSucc(IcebergRestTestUtil.TEST_NAMESPACE_NAME); verifyCreateViewSucc("list_foo1"); verifyCreateViewSucc("list_foo2"); + + dummyEventListener.clearEvent(); verifyLisViewSucc(ImmutableSet.of("list_foo1", "list_foo2")); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergListViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergListViewEvent); } @Test void testCreateView() { verifyCreateViewFail("create_foo1", 404); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergCreateViewPreEvent); + Assertions.assertTrue( + dummyEventListener.popPostEvent() instanceof IcebergCreateViewFailureEvent); verifyCreateNamespaceSucc(IcebergRestTestUtil.TEST_NAMESPACE_NAME); verifyCreateViewSucc("create_foo1"); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergCreateViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergCreateViewEvent); verifyCreateViewFail("create_foo1", 409); verifyCreateViewFail("", 400); @@ -94,10 +131,16 @@ void testCreateView() { @Test void testLoadView() { verifyLoadViewFail("load_foo1", 404); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergLoadViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergLoadViewFailureEvent); verifyCreateNamespaceSucc(IcebergRestTestUtil.TEST_NAMESPACE_NAME); verifyCreateViewSucc("load_foo1"); + + dummyEventListener.clearEvent(); verifyLoadViewSucc("load_foo1"); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergLoadViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergLoadViewEvent); verifyLoadViewFail("load_foo2", 404); } @@ -107,10 +150,19 @@ void testReplaceView() { verifyCreateNamespaceSucc(IcebergRestTestUtil.TEST_NAMESPACE_NAME); verifyCreateViewSucc("replace_foo1"); ViewMetadata metadata = getViewMeta("replace_foo1"); + + dummyEventListener.clearEvent(); verifyReplaceSucc("replace_foo1", metadata); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergReplaceViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergReplaceViewEvent); verifyDropViewSucc("replace_foo1"); + + dummyEventListener.clearEvent(); verifyUpdateViewFail("replace_foo1", 404, metadata); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergReplaceViewPreEvent); + Assertions.assertTrue( + dummyEventListener.popPostEvent() instanceof IcebergReplaceViewFailureEvent); verifyDropNamespaceSucc(IcebergRestTestUtil.TEST_NAMESPACE_NAME); verifyUpdateViewFail("replace_foo1", 404, metadata); @@ -119,22 +171,41 @@ void testReplaceView() { @Test void testDropView() { verifyDropViewFail("drop_foo1", 404); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergDropViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergDropViewFailureEvent); + verifyCreateNamespaceSucc(IcebergRestTestUtil.TEST_NAMESPACE_NAME); verifyDropViewFail("drop_foo1", 404); verifyCreateViewSucc("drop_foo1"); + + dummyEventListener.clearEvent(); verifyDropViewSucc("drop_foo1"); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergDropViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergDropViewEvent); + verifyLoadViewFail("drop_foo1", 404); } @Test void testViewExits() { verifyViewExistsStatusCode("exists_foo2", 404); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergViewExistsPreEvent); + Event postEvent = dummyEventListener.popPostEvent(); + Assertions.assertTrue(postEvent instanceof IcebergViewExistsEvent); + Assertions.assertEquals(false, ((IcebergViewExistsEvent) postEvent).isExists()); + verifyCreateNamespaceSucc(IcebergRestTestUtil.TEST_NAMESPACE_NAME); verifyViewExistsStatusCode("exists_foo2", 404); verifyCreateViewSucc("exists_foo1"); + dummyEventListener.clearEvent(); verifyViewExistsStatusCode("exists_foo1", 204); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergViewExistsPreEvent); + postEvent = dummyEventListener.popPostEvent(); + Assertions.assertTrue(postEvent instanceof IcebergViewExistsEvent); + Assertions.assertEquals(true, ((IcebergViewExistsEvent) postEvent).isExists()); + verifyLoadViewSucc("exists_foo1"); } @@ -144,11 +215,19 @@ void testRenameTable(String prefix) { setUrlPathWithPrefix(prefix); // namespace not exits verifyRenameViewFail("rename_foo1", "rename_foo3", 404); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergRenameViewPreEvent); + Assertions.assertTrue( + dummyEventListener.popPostEvent() instanceof IcebergRenameViewFailureEvent); verifyCreateNamespaceSucc(IcebergRestTestUtil.TEST_NAMESPACE_NAME); verifyCreateViewSucc("rename_foo1"); + + dummyEventListener.clearEvent(); // rename verifyRenameViewSucc("rename_foo1", "rename_foo2"); + Assertions.assertTrue(dummyEventListener.popPreEvent() instanceof IcebergRenameViewPreEvent); + Assertions.assertTrue(dummyEventListener.popPostEvent() instanceof IcebergRenameViewEvent); + verifyLoadViewFail("rename_foo1", 404); verifyLoadViewSucc("rename_foo2");