From b62dfbc3733d437aafb7e78ab8d6a35c9c06046a Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Mon, 14 Jul 2025 17:24:25 -0700 Subject: [PATCH 01/17] merge from main --- ...olarisEclipseLinkMetaStoreManagerTest.java | 3 +- persistence/relational-jdbc/build.gradle.kts | 3 + .../relational/jdbc/DatabaseType.java | 3 +- .../relational/jdbc/DatasourceOperations.java | 59 +++++ .../jdbc/JdbcBasePersistenceImpl.java | 56 ++++ .../relational/jdbc/QueryGenerator.java | 3 + .../relational/jdbc/models/ModelEvent.java | 143 ++++++++++ .../src/main/resources/h2/schema-v1.sql | 14 + .../src/main/resources/h2/schema-v3.sql | 140 ++++++++++ .../src/main/resources/postgres/schema-v3.sql | 137 ++++++++++ ...anagerWithJdbcBasePersistenceImplTest.java | 3 +- .../jdbc/DatasourceOperationsTest.java | 36 +++ .../jdbc/models/ModelEventTest.java | 249 ++++++++++++++++++ .../polaris/core/PolarisCallContext.java | 26 +- .../polaris/core/entity/PolarisEvent.java | 137 ++++++++++ .../core/entity/PolarisEventManager.java | 34 +++ .../AtomicOperationMetaStoreManager.java | 10 + .../core/persistence/BasePersistence.java | 8 + .../persistence/PolarisMetaStoreManager.java | 4 +- .../TransactionWorkspaceMetaStoreManager.java | 9 + .../AbstractTransactionalPersistence.java | 6 + ...apAtomicOperationMetaStoreManagerTest.java | 3 +- .../PolarisTreeMapMetaStoreManagerTest.java | 3 +- .../InMemoryStorageIntegrationTest.java | 3 +- .../polaris/admintool/BootstrapCommand.java | 2 +- .../src/main/resources/application.properties | 5 + .../config/ProductionReadinessChecks.java | 6 +- .../quarkus/config/QuarkusProducers.java | 9 +- ...rkusPolarisEventListenerConfiguration.java | 17 +- .../quarkus/task/QuarkusTaskExecutorImpl.java | 2 +- .../quarkus/admin/ManagementServiceTest.java | 3 +- .../quarkus/admin/PolarisAuthzTestBase.java | 9 +- .../quarkus/auth/JWTRSAKeyPairTest.java | 2 +- .../auth/JWTSymmetricKeyGeneratorTest.java | 2 +- .../IcebergCatalogHandlerAuthzTest.java | 9 +- .../quarkus/catalog/IcebergCatalogTest.java | 9 +- .../catalog/IcebergCatalogViewTest.java | 7 +- .../PolarisGenericTableCatalogTest.java | 5 +- .../quarkus/catalog/PolicyCatalogTest.java | 5 +- .../ratelimiter/RateLimiterFilterTest.java | 4 +- .../task/TableCleanupTaskHandlerTest.java | 3 +- .../test/PolarisIntegrationTestFixture.java | 3 +- .../test/PolarisIntegrationTestHelper.java | 2 + service/common/build.gradle.kts | 1 + .../service/admin/PolarisServiceImpl.java | 12 +- .../catalog/iceberg/IcebergCatalog.java | 41 ++- .../iceberg/IcebergCatalogAdapter.java | 9 +- .../iceberg/IcebergCatalogHandler.java | 43 ++- .../context/DefaultCallContextResolver.java | 2 +- .../PolarisCallContextCatalogFactory.java | 2 +- .../events/AfterCatalogCreatedEvent.java | 24 ++ .../events/AfterTableCommitedEvent.java | 2 +- .../events/AfterTableCreatedEvent.java | 28 ++ .../events/AfterTableRefreshedEvent.java | 3 +- .../events/AfterTaskAttemptedEvent.java | 5 +- .../events/AfterViewCommitedEvent.java | 3 +- .../events/AfterViewRefreshedEvent.java | 2 +- .../events/BeforeRequestRateLimitedEvent.java | 2 +- .../events/BeforeTableCommitedEvent.java | 4 +- .../events/BeforeTableRefreshedEvent.java | 2 +- .../events/BeforeTaskAttemptedEvent.java | 3 +- .../events/BeforeViewCommitedEvent.java | 2 +- .../events/BeforeViewRefreshedEvent.java | 2 +- .../events/EventListenerConfiguration.java | 41 +++ .../polaris/service/events/PolarisEvent.java | 12 +- .../service/events/PolarisEventListener.java | 58 ---- .../events/TestPolarisEventListener.java | 92 ------- ...BufferPolarisPersistenceEventListener.java | 144 ++++++++++ .../NoOpPolarisEventListener.java | 2 +- .../listeners/PolarisEventListener.java | 92 +++++++ .../PolarisPersistenceEventListener.java | 139 ++++++++++ .../listeners/TestPolarisEventListener.java | 115 ++++++++ .../ratelimiter/RateLimiterFilter.java | 10 +- .../service/task/TaskExecutorImpl.java | 10 +- .../service/admin/PolarisServiceImplTest.java | 7 +- .../service/catalog/io/FileIOFactoryTest.java | 3 +- ...erPolarisPersistenceEventListenerTest.java | 197 ++++++++++++++ .../service/task/TaskExecutorImplTest.java | 5 +- .../apache/polaris/service/TestServices.java | 13 +- 79 files changed, 2073 insertions(+), 255 deletions(-) create mode 100644 persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java create mode 100644 persistence/relational-jdbc/src/main/resources/h2/schema-v3.sql create mode 100644 persistence/relational-jdbc/src/main/resources/postgres/schema-v3.sql create mode 100644 persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java create mode 100644 polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java create mode 100644 polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEventManager.java create mode 100644 service/common/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java create mode 100644 service/common/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java create mode 100644 service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java delete mode 100644 service/common/src/main/java/org/apache/polaris/service/events/PolarisEventListener.java delete mode 100644 service/common/src/main/java/org/apache/polaris/service/events/TestPolarisEventListener.java create mode 100644 service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java rename service/common/src/main/java/org/apache/polaris/service/events/{ => listeners}/NoOpPolarisEventListener.java (95%) create mode 100644 service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java create mode 100644 service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java create mode 100644 service/common/src/main/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java create mode 100644 service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java diff --git a/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java b/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java index 55607981e9..acfd6cf89c 100644 --- a/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java +++ b/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java @@ -96,7 +96,8 @@ protected PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { session, diagServices, new PolarisConfigurationStore() {}, - timeSource.withZone(ZoneId.systemDefault()))); + timeSource.withZone(ZoneId.systemDefault()), + null)); } @ParameterizedTest diff --git a/persistence/relational-jdbc/build.gradle.kts b/persistence/relational-jdbc/build.gradle.kts index 2750d4e193..a260ba9443 100644 --- a/persistence/relational-jdbc/build.gradle.kts +++ b/persistence/relational-jdbc/build.gradle.kts @@ -36,6 +36,9 @@ dependencies { implementation(libs.smallrye.common.annotation) // @Identifier implementation(libs.postgresql) + compileOnly(project(":polaris-immutables")) + annotationProcessor(project(":polaris-immutables", configuration = "processor")) + testImplementation(libs.mockito.junit.jupiter) testImplementation(libs.h2) testImplementation(testFixtures(project(":polaris-core"))) diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/DatabaseType.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/DatabaseType.java index f731eb5508..c617e505e6 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/DatabaseType.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/DatabaseType.java @@ -62,9 +62,10 @@ public InputStream openInitScriptResource(@Nonnull SchemaOptions schemaOptions) } else { final String schemaSuffix; switch (schemaOptions.schemaVersion()) { - case null -> schemaSuffix = "schema-v2.sql"; + case null -> schemaSuffix = "schema-v3.sql"; case 1 -> schemaSuffix = "schema-v1.sql"; case 2 -> schemaSuffix = "schema-v2.sql"; + case 3 -> schemaSuffix = "schema-v3.sql"; default -> throw new IllegalArgumentException( "Unknown schema version " + schemaOptions.schemaVersion()); diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperations.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperations.java index 8dd9948698..51feba7d1c 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperations.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperations.java @@ -32,11 +32,13 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Objects; import java.util.Random; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.stream.Stream; import javax.sql.DataSource; @@ -193,6 +195,63 @@ public int executeUpdate(QueryGenerator.PreparedQuery preparedQuery) throws SQLE }); } + /** + * Executes the INSERT/UPDATE Queries in batches. Requires that all SQL queries have the same + * parameterized form. + * + * @param preparedQueries : queries to be executed + * @return : Number of rows modified / inserted. + * @throws SQLException : Exception during Query Execution. + */ + public int executeBatchUpdate(QueryGenerator.PreparedBatchQuery preparedQueries) + throws SQLException { + if (preparedQueries.parametersList().isEmpty() || preparedQueries.sql().isEmpty()) { + return 0; + } + int batchSize = 100; + AtomicInteger successCount = new AtomicInteger(); + return withRetries( + () -> { + try (Connection connection = borrowConnection(); + PreparedStatement statement = connection.prepareStatement(preparedQueries.sql())) { + boolean autoCommit = connection.getAutoCommit(); + boolean success = false; + connection.setAutoCommit(false); + + try { + for (int i = 1; i <= preparedQueries.parametersList().size(); i++) { + List params = preparedQueries.parametersList().get(i - 1); + for (int j = 0; j < params.size(); j++) { + statement.setObject(j + 1, params.get(j)); + } + + statement.addBatch(); // Add to batch + + if (i % batchSize == 0) { + successCount.addAndGet(Arrays.stream(statement.executeBatch()).sum()); + } + } + + // Execute remaining queries in the batch + successCount.addAndGet(Arrays.stream(statement.executeBatch()).sum()); + success = true; + } finally { + try { + if (success) { + connection.commit(); + } else { + connection.rollback(); + successCount.set(0); + } + } finally { + connection.setAutoCommit(autoCommit); + } + } + } + return successCount.get(); + }); + } + /** * Transaction callback to be executed. * diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java index 5c3dd1dbaf..9ac1c38851 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java @@ -44,6 +44,7 @@ import org.apache.polaris.core.entity.PolarisEntityCore; import org.apache.polaris.core.entity.PolarisEntityId; import org.apache.polaris.core.entity.PolarisEntityType; +import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.entity.PolarisGrantRecord; import org.apache.polaris.core.entity.PolarisPrincipalSecrets; import org.apache.polaris.core.persistence.BaseMetaStoreManager; @@ -64,6 +65,7 @@ import org.apache.polaris.core.storage.PolarisStorageIntegrationProvider; import org.apache.polaris.core.storage.StorageLocation; import org.apache.polaris.persistence.relational.jdbc.models.ModelEntity; +import org.apache.polaris.persistence.relational.jdbc.models.ModelEvent; import org.apache.polaris.persistence.relational.jdbc.models.ModelGrantRecord; import org.apache.polaris.persistence.relational.jdbc.models.ModelPolicyMappingRecord; import org.apache.polaris.persistence.relational.jdbc.models.ModelPrincipalAuthenticationData; @@ -235,6 +237,60 @@ public void writeToGrantRecords( } } + @Override + public void writeEvents(@Nonnull List events) { + if (events.isEmpty()) { + return; // or throw if empty list is invalid + } + + try { + // Generate the SQL using the first event as the reference + PreparedQuery firstPreparedQuery = QueryGenerator.generateInsertQuery( + ModelEvent.ALL_COLUMNS, + ModelEvent.TABLE_NAME, + ModelEvent.fromEvent(events.get(0)) + .toMap(datasourceOperations.getDatabaseType()) + .values() + .stream() + .toList(), + realmId); + String expectedSql = firstPreparedQuery.sql(); + + List> parametersList = new ArrayList<>(); + parametersList.add(firstPreparedQuery.parameters()); + + // Process remaining events and verify SQL consistency + for (int i = 1; i < events.size(); i++) { + PolarisEvent event = events.get(i); + PreparedQuery pq = QueryGenerator.generateInsertQuery( + ModelEvent.ALL_COLUMNS, + ModelEvent.TABLE_NAME, + ModelEvent.fromEvent(event) + .toMap(datasourceOperations.getDatabaseType()) + .values() + .stream() + .toList(), + realmId); + + if (!expectedSql.equals(pq.sql())) { + throw new RuntimeException("All events did not generate the same SQL"); + } + + parametersList.add(pq.parameters()); + } + + int totalUpdated = datasourceOperations.executeBatchUpdate( + new QueryGenerator.PreparedBatchQuery(expectedSql, parametersList)); + + if (totalUpdated == 0) { + throw new SQLException("No events were inserted."); + } + } catch (SQLException e) { + throw new RuntimeException( + String.format("Failed to write events due to %s", e.getMessage()), e); + } + } + @Override public void deleteEntity(@Nonnull PolarisCallContext callCtx, @Nonnull PolarisBaseEntity entity) { ModelEntity modelEntity = ModelEntity.fromEntity(entity); diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/QueryGenerator.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/QueryGenerator.java index c6bad0a1c5..076d5aa00d 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/QueryGenerator.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/QueryGenerator.java @@ -43,6 +43,9 @@ public class QueryGenerator { /** A container for the SQL string and the ordered parameter values. */ public record PreparedQuery(String sql, List parameters) {} + /** A container for the SQL string and a list of the ordered parameter values. */ + public record PreparedBatchQuery(String sql, List> parametersList) {} + /** A container for the query fragment SQL string and the ordered parameter values. */ record QueryFragment(String sql, List parameters) {} diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java new file mode 100644 index 0000000000..a637c58ea6 --- /dev/null +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.polaris.persistence.relational.jdbc.models; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.immutables.PolarisImmutable; +import org.apache.polaris.persistence.relational.jdbc.DatabaseType; + +@PolarisImmutable +public interface ModelEvent extends Converter { + String TABLE_NAME = "EVENTS"; + + List ALL_COLUMNS = + List.of( + "catalog_id", + "event_id", + "request_id", + "event_type", + "timestamp_ms", + "principal_name", + "resource_type", + "resource_identifier", + "additional_parameters"); + + // catalog id + String getCatalogId(); + + // event id + String getEventId(); + + // id of the request that generated this event + String getRequestId(); + + // event type that was created + String getEventType(); + + // timestamp in epoch milliseconds of when this event was emitted + long getTimestampMs(); + + // polaris principal who took this action + String getPrincipalName(); + + // Enum that states the type of resource was being operated on + PolarisEvent.ResourceType getResourceType(); + + // Which resource was operated on + String getResourceIdentifier(); + + // Additional parameters that were not earlier recorded + String getAdditionalParameters(); + + @Override + default PolarisEvent fromResultSet(ResultSet rs) throws SQLException { + var modelEvent = + ImmutableModelEvent.builder() + .catalogId(rs.getString("catalog_id")) + .eventId(rs.getString("event_id")) + .requestId(rs.getString("request_id")) + .eventType(rs.getString("event_type")) + .timestampMs(rs.getLong("timestamp_ms")) + .principalName(rs.getString("actor")) + .resourceType(PolarisEvent.ResourceType.valueOf(rs.getString("resource_type"))) + .resourceIdentifier(rs.getString("resource_identifier")) + .additionalParameters(rs.getString("additional_parameters")) + .build(); + return toEvent(modelEvent); + } + + @Override + default Map toMap(DatabaseType databaseType) { + Map map = new LinkedHashMap<>(); + map.put("catalog_id", getCatalogId()); + map.put("event_id", getEventId()); + map.put("request_id", getRequestId()); + map.put("event_type", getEventType()); + map.put("timestamp_ms", getTimestampMs()); + map.put("principal_name", getPrincipalName()); + map.put("resource_type", getResourceType().toString()); + map.put("resource_identifier", getResourceIdentifier()); + if (databaseType.equals(DatabaseType.POSTGRES)) { + map.put("additional_parameters", toJsonbPGobject(getAdditionalParameters())); + } else { + map.put("additional_parameters", getAdditionalParameters()); + } + return map; + } + + static ModelEvent fromEvent(PolarisEvent event) { + if (event == null) return null; + + return ImmutableModelEvent.builder() + .catalogId(event.getCatalogId()) + .eventId(event.getId()) + .requestId(event.getRequestId()) + .eventType(event.getEventType()) + .timestampMs(event.getTimestampMs()) + .principalName(event.getPrincipalName()) + .resourceType(event.getResourceType()) + .resourceIdentifier(event.getResourceIdentifier()) + .additionalParameters(event.getAdditionalParameters()) + .build(); + } + + static PolarisEvent toEvent(ModelEvent model) { + if (model == null) return null; + + PolarisEvent polarisEvent = + new PolarisEvent( + model.getCatalogId(), + model.getEventId(), + model.getRequestId(), + model.getEventType(), + model.getTimestampMs(), + model.getPrincipalName(), + model.getResourceType(), + model.getResourceIdentifier()); + polarisEvent.setAdditionalParameters(model.getAdditionalParameters()); + return polarisEvent; + } +} diff --git a/persistence/relational-jdbc/src/main/resources/h2/schema-v1.sql b/persistence/relational-jdbc/src/main/resources/h2/schema-v1.sql index 10b4e3774b..734462be91 100644 --- a/persistence/relational-jdbc/src/main/resources/h2/schema-v1.sql +++ b/persistence/relational-jdbc/src/main/resources/h2/schema-v1.sql @@ -118,3 +118,17 @@ CREATE TABLE IF NOT EXISTS policy_mapping_record ( ); CREATE INDEX IF NOT EXISTS idx_policy_mapping_record ON policy_mapping_record (realm_id, policy_type_code, policy_catalog_id, policy_id, target_catalog_id, target_id); + +CREATE TABLE IF NOT EXISTS events ( + realm_id TEXT NOT NULL, + catalog_id TEXT NOT NULL, + event_id TEXT NOT NULL, + request_id TEXT NOT NULL, + event_type TEXT NOT NULL, + timestamp_ms BIGINT NOT NULL, + principal_name TEXT, + resource_type TEXT NOT NULL, + resource_identifier TEXT NOT NULL, + additional_parameters TEXT NOT NULL DEFAULT '{}', + PRIMARY KEY (event_id) +); diff --git a/persistence/relational-jdbc/src/main/resources/h2/schema-v3.sql b/persistence/relational-jdbc/src/main/resources/h2/schema-v3.sql new file mode 100644 index 0000000000..5bdf52a565 --- /dev/null +++ b/persistence/relational-jdbc/src/main/resources/h2/schema-v3.sql @@ -0,0 +1,140 @@ +-- +-- 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. +-- + +-- Changes from v2: +-- * Added `events` table + +CREATE SCHEMA IF NOT EXISTS POLARIS_SCHEMA; +SET SCHEMA POLARIS_SCHEMA; + +CREATE TABLE IF NOT EXISTS version ( + version_key VARCHAR PRIMARY KEY, + version_value INTEGER NOT NULL +); + +MERGE INTO version (version_key, version_value) + KEY (version_key) + VALUES ('version', 2); + +-- H2 supports COMMENT, but some modes may ignore it +COMMENT ON TABLE version IS 'the version of the JDBC schema in use'; + +DROP TABLE IF EXISTS entities; +CREATE TABLE IF NOT EXISTS entities ( + realm_id TEXT NOT NULL, + catalog_id BIGINT NOT NULL, + id BIGINT NOT NULL, + parent_id BIGINT NOT NULL, + name TEXT NOT NULL, + entity_version INT NOT NULL, + type_code INT NOT NULL, + sub_type_code INT NOT NULL, + create_timestamp BIGINT NOT NULL, + drop_timestamp BIGINT NOT NULL, + purge_timestamp BIGINT NOT NULL, + to_purge_timestamp BIGINT NOT NULL, + last_update_timestamp BIGINT NOT NULL, + properties TEXT NOT NULL DEFAULT '{}', + internal_properties TEXT NOT NULL DEFAULT '{}', + grant_records_version INT NOT NULL, + location_without_scheme TEXT, + PRIMARY KEY (realm_id, id), + CONSTRAINT constraint_name UNIQUE (realm_id, catalog_id, parent_id, type_code, name) +); + +CREATE INDEX IF NOT EXISTS idx_locations ON entities(realm_id, catalog_id, location_without_scheme); + +-- TODO: create indexes based on all query pattern. +CREATE INDEX IF NOT EXISTS idx_entities ON entities (realm_id, catalog_id, id); + +COMMENT ON TABLE entities IS 'all the entities'; + +COMMENT ON COLUMN entities.catalog_id IS 'catalog id'; +COMMENT ON COLUMN entities.id IS 'entity id'; +COMMENT ON COLUMN entities.parent_id IS 'entity id of parent'; +COMMENT ON COLUMN entities.name IS 'entity name'; +COMMENT ON COLUMN entities.entity_version IS 'version of the entity'; +COMMENT ON COLUMN entities.type_code IS 'type code'; +COMMENT ON COLUMN entities.sub_type_code IS 'sub type of entity'; +COMMENT ON COLUMN entities.create_timestamp IS 'creation time of entity'; +COMMENT ON COLUMN entities.drop_timestamp IS 'time of drop of entity'; +COMMENT ON COLUMN entities.purge_timestamp IS 'time to start purging entity'; +COMMENT ON COLUMN entities.last_update_timestamp IS 'last time the entity is touched'; +COMMENT ON COLUMN entities.properties IS 'entities properties json'; +COMMENT ON COLUMN entities.internal_properties IS 'entities internal properties json'; +COMMENT ON COLUMN entities.grant_records_version IS 'the version of grant records change on the entity'; + +DROP TABLE IF EXISTS grant_records; +CREATE TABLE IF NOT EXISTS grant_records ( + realm_id TEXT NOT NULL, + securable_catalog_id BIGINT NOT NULL, + securable_id BIGINT NOT NULL, + grantee_catalog_id BIGINT NOT NULL, + grantee_id BIGINT NOT NULL, + privilege_code INTEGER, + PRIMARY KEY (realm_id, securable_catalog_id, securable_id, grantee_catalog_id, grantee_id, privilege_code) +); + +COMMENT ON TABLE grant_records IS 'grant records for entities'; +COMMENT ON COLUMN grant_records.securable_catalog_id IS 'catalog id of the securable'; +COMMENT ON COLUMN grant_records.securable_id IS 'entity id of the securable'; +COMMENT ON COLUMN grant_records.grantee_catalog_id IS 'catalog id of the grantee'; +COMMENT ON COLUMN grant_records.grantee_id IS 'id of the grantee'; +COMMENT ON COLUMN grant_records.privilege_code IS 'privilege code'; + +DROP TABLE IF EXISTS principal_authentication_data; +CREATE TABLE IF NOT EXISTS principal_authentication_data ( + realm_id TEXT NOT NULL, + principal_id BIGINT NOT NULL, + principal_client_id VARCHAR(255) NOT NULL, + main_secret_hash VARCHAR(255) NOT NULL, + secondary_secret_hash VARCHAR(255) NOT NULL, + secret_salt VARCHAR(255) NOT NULL, + PRIMARY KEY (realm_id, principal_client_id) +); + +COMMENT ON TABLE principal_authentication_data IS 'authentication data for client'; + +DROP TABLE IF EXISTS policy_mapping_record; +CREATE TABLE IF NOT EXISTS policy_mapping_record ( + realm_id TEXT NOT NULL, + target_catalog_id BIGINT NOT NULL, + target_id BIGINT NOT NULL, + policy_type_code INTEGER NOT NULL, + policy_catalog_id BIGINT NOT NULL, + policy_id BIGINT NOT NULL, + parameters TEXT NOT NULL DEFAULT '{}', + PRIMARY KEY (realm_id, target_catalog_id, target_id, policy_type_code, policy_catalog_id, policy_id) +); + +CREATE INDEX IF NOT EXISTS idx_policy_mapping_record ON policy_mapping_record (realm_id, policy_type_code, policy_catalog_id, policy_id, target_catalog_id, target_id); + +CREATE TABLE IF NOT EXISTS events ( + realm_id TEXT NOT NULL, + catalog_id TEXT NOT NULL, + event_id TEXT NOT NULL, + request_id TEXT NOT NULL, + event_type TEXT NOT NULL, + timestamp_ms BIGINT NOT NULL, + principal_name TEXT, + resource_type TEXT NOT NULL, + resource_identifier TEXT NOT NULL, + additional_parameters JSONB NOT NULL DEFAULT '{}'::JSONB, + PRIMARY KEY (event_id) +); \ No newline at end of file diff --git a/persistence/relational-jdbc/src/main/resources/postgres/schema-v3.sql b/persistence/relational-jdbc/src/main/resources/postgres/schema-v3.sql new file mode 100644 index 0000000000..628eef84b4 --- /dev/null +++ b/persistence/relational-jdbc/src/main/resources/postgres/schema-v3.sql @@ -0,0 +1,137 @@ +-- +-- 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. + +-- Changes from v2: +-- * Added `events` table + +CREATE SCHEMA IF NOT EXISTS POLARIS_SCHEMA; +SET search_path TO POLARIS_SCHEMA; + +CREATE TABLE IF NOT EXISTS version ( + version_key TEXT PRIMARY KEY, + version_value INTEGER NOT NULL +); +INSERT INTO version (version_key, version_value) +VALUES ('version', 3) +ON CONFLICT (version_key) DO UPDATE +SET version_value = EXCLUDED.version_value; +COMMENT ON TABLE version IS 'the version of the JDBC schema in use'; + +CREATE TABLE IF NOT EXISTS entities ( + realm_id TEXT NOT NULL, + catalog_id BIGINT NOT NULL, + id BIGINT NOT NULL, + parent_id BIGINT NOT NULL, + name TEXT NOT NULL, + entity_version INT NOT NULL, + type_code INT NOT NULL, + sub_type_code INT NOT NULL, + create_timestamp BIGINT NOT NULL, + drop_timestamp BIGINT NOT NULL, + purge_timestamp BIGINT NOT NULL, + to_purge_timestamp BIGINT NOT NULL, + last_update_timestamp BIGINT NOT NULL, + properties JSONB not null default '{}'::JSONB, + internal_properties JSONB not null default '{}'::JSONB, + grant_records_version INT NOT NULL, + location_without_scheme TEXT, + PRIMARY KEY (realm_id, id), + CONSTRAINT constraint_name UNIQUE (realm_id, catalog_id, parent_id, type_code, name) +); + +-- TODO: create indexes based on all query pattern. +CREATE INDEX IF NOT EXISTS idx_entities ON entities (realm_id, catalog_id, id); +CREATE INDEX IF NOT EXISTS idx_locations + ON entities USING btree (realm_id, parent_id, location_without_scheme) + WHERE location_without_scheme IS NOT NULL; + +COMMENT ON TABLE entities IS 'all the entities'; + +COMMENT ON COLUMN entities.realm_id IS 'realm_id used for multi-tenancy'; +COMMENT ON COLUMN entities.catalog_id IS 'catalog id'; +COMMENT ON COLUMN entities.id IS 'entity id'; +COMMENT ON COLUMN entities.parent_id IS 'entity id of parent'; +COMMENT ON COLUMN entities.name IS 'entity name'; +COMMENT ON COLUMN entities.entity_version IS 'version of the entity'; +COMMENT ON COLUMN entities.type_code IS 'type code'; +COMMENT ON COLUMN entities.sub_type_code IS 'sub type of entity'; +COMMENT ON COLUMN entities.create_timestamp IS 'creation time of entity'; +COMMENT ON COLUMN entities.drop_timestamp IS 'time of drop of entity'; +COMMENT ON COLUMN entities.purge_timestamp IS 'time to start purging entity'; +COMMENT ON COLUMN entities.last_update_timestamp IS 'last time the entity is touched'; +COMMENT ON COLUMN entities.properties IS 'entities properties json'; +COMMENT ON COLUMN entities.internal_properties IS 'entities internal properties json'; +COMMENT ON COLUMN entities.grant_records_version IS 'the version of grant records change on the entity'; + +CREATE TABLE IF NOT EXISTS grant_records ( + realm_id TEXT NOT NULL, + securable_catalog_id BIGINT NOT NULL, + securable_id BIGINT NOT NULL, + grantee_catalog_id BIGINT NOT NULL, + grantee_id BIGINT NOT NULL, + privilege_code INTEGER, + PRIMARY KEY (realm_id, securable_catalog_id, securable_id, grantee_catalog_id, grantee_id, privilege_code) +); + +COMMENT ON TABLE grant_records IS 'grant records for entities'; + +COMMENT ON COLUMN grant_records.securable_catalog_id IS 'catalog id of the securable'; +COMMENT ON COLUMN grant_records.securable_id IS 'entity id of the securable'; +COMMENT ON COLUMN grant_records.grantee_catalog_id IS 'catalog id of the grantee'; +COMMENT ON COLUMN grant_records.grantee_id IS 'id of the grantee'; +COMMENT ON COLUMN grant_records.privilege_code IS 'privilege code'; + +CREATE TABLE IF NOT EXISTS principal_authentication_data ( + realm_id TEXT NOT NULL, + principal_id BIGINT NOT NULL, + principal_client_id VARCHAR(255) NOT NULL, + main_secret_hash VARCHAR(255) NOT NULL, + secondary_secret_hash VARCHAR(255) NOT NULL, + secret_salt VARCHAR(255) NOT NULL, + PRIMARY KEY (realm_id, principal_client_id) +); + +COMMENT ON TABLE principal_authentication_data IS 'authentication data for client'; + +DROP TABLE IF EXISTS policy_mapping_record; +CREATE TABLE IF NOT EXISTS policy_mapping_record ( + realm_id TEXT NOT NULL, + target_catalog_id BIGINT NOT NULL, + target_id BIGINT NOT NULL, + policy_type_code INTEGER NOT NULL, + policy_catalog_id BIGINT NOT NULL, + policy_id BIGINT NOT NULL, + parameters JSONB NOT NULL DEFAULT '{}'::JSONB, + PRIMARY KEY (realm_id, target_catalog_id, target_id, policy_type_code, policy_catalog_id, policy_id) +); + +CREATE INDEX IF NOT EXISTS idx_policy_mapping_record ON policy_mapping_record (realm_id, policy_type_code, policy_catalog_id, policy_id, target_catalog_id, target_id); + +CREATE TABLE IF NOT EXISTS events ( + realm_id TEXT NOT NULL, + catalog_id TEXT NOT NULL, + event_id TEXT NOT NULL, + request_id TEXT NOT NULL, + event_type TEXT NOT NULL, + timestamp_ms BIGINT NOT NULL, + principal_name TEXT, + resource_type TEXT NOT NULL, + resource_identifier TEXT NOT NULL, + additional_parameters JSONB NOT NULL DEFAULT '{}'::JSONB, + PRIMARY KEY (event_id) +); diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/AtomicMetastoreManagerWithJdbcBasePersistenceImplTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/AtomicMetastoreManagerWithJdbcBasePersistenceImplTest.java index bbb57e68ba..be5cfceef9 100644 --- a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/AtomicMetastoreManagerWithJdbcBasePersistenceImplTest.java +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/AtomicMetastoreManagerWithJdbcBasePersistenceImplTest.java @@ -76,7 +76,8 @@ protected PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { basePersistence, diagServices, new PolarisConfigurationStore() {}, - timeSource.withZone(ZoneId.systemDefault()))); + timeSource.withZone(ZoneId.systemDefault()), + null)); } private static class H2JdbcConfiguration implements RelationalJdbcConfiguration { diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java index ae6f569cc5..b4416e2109 100644 --- a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java @@ -21,6 +21,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.atMost; import static org.mockito.Mockito.reset; @@ -33,11 +34,15 @@ import java.sql.PreparedStatement; import java.sql.SQLException; import java.time.Instant; +import java.util.ArrayList; import java.util.List; import java.util.Optional; import javax.sql.DataSource; +import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.persistence.relational.jdbc.DatasourceOperations.Operation; +import org.apache.polaris.persistence.relational.jdbc.models.ImmutableModelEvent; import org.apache.polaris.persistence.relational.jdbc.models.ModelEntity; +import org.apache.polaris.persistence.relational.jdbc.models.ModelEvent; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -92,6 +97,37 @@ void testExecuteUpdate_failure() throws Exception { assertThrows(SQLException.class, () -> datasourceOperations.executeUpdate(query)); } + @Test + void executeBatchUpdate_success() throws Exception { + // There is no way to track how many statements are in a batch, so we are testing how many times + // `executeBatch` is being called + when(mockDataSource.getConnection()).thenReturn(mockConnection); + String sql = "INSERT INTO POLARIS_SCHEMA.EVENTS (catalog_id, event_id, request_id, event_type, timestamp_ms, principal_name, resource_type, resource_identifier, additional_parameters, realm_id) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ?, ? )"; + List> queryParams = new ArrayList<>(); + for (int i = 0; i < 1000; i++) { + ModelEvent modelEvent = + ImmutableModelEvent.builder() + .resourceType(PolarisEvent.ResourceType.CATALOG) + .resourceIdentifier("catalog_" + i) + .catalogId("catalog_" + i) + .eventId("event_" + i) + .requestId("request_" + i) + .eventType("event_type1") + .timestampMs(1234) + .principalName("principal_" + i) + .additionalParameters("") + .build(); + queryParams.add( + modelEvent.toMap(datasourceOperations.getDatabaseType()).values().stream().toList()); + } + when(mockConnection.prepareStatement(any())).thenReturn(mockPreparedStatement); + when(mockPreparedStatement.executeBatch()).thenReturn(new int[] {100}); + + int result = datasourceOperations.executeBatchUpdate(new QueryGenerator.PreparedBatchQuery(sql, queryParams)); + assertEquals( + queryParams.size() + 100, result); // ExecuteBatch will be called once more than actual batches + } + @Test void testExecuteSelect_exception() throws Exception { when(mockDataSource.getConnection()).thenReturn(mockConnection); diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java new file mode 100644 index 0000000000..d0b5a21112 --- /dev/null +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java @@ -0,0 +1,249 @@ +/* + * 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.polaris.persistence.relational.jdbc.models; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Map; + +import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.persistence.relational.jdbc.DatabaseType; +import org.junit.jupiter.api.Test; +import org.postgresql.util.PGobject; + +public class ModelEventTest { + // Column names + private static final String CATALOG_ID = "catalog_id"; + private static final String EVENT_ID = "event_id"; + private static final String REQUEST_ID = "request_id"; + private static final String EVENT_TYPE = "event_type"; + private static final String TIMESTAMP_MS = "timestamp_ms"; + private static final String ACTOR = "actor"; + private static final String PRINCIPAL_NAME = "principal_name"; + private static final String RESOURCE_TYPE = "resource_type"; + private static final String RESOURCE_IDENTIFIER = "resource_identifier"; + private static final String ADDITIONAL_PARAMETERS = "additional_parameters"; + + // Test data values + private static final String TEST_CATALOG_ID = "test-catalog"; + private static final String TEST_EVENT_ID = "event-123"; + private static final String TEST_REQUEST_ID = "req-456"; + private static final String TEST_EVENT_TYPE = "CREATE"; + private static final long TEST_TIMESTAMP_MS = 1234567890L; + private static final String TEST_USER = "test-user"; + private static final PolarisEvent.ResourceType TEST_RESOURCE_TYPE = PolarisEvent.ResourceType.TABLE; + private static final String TEST_RESOURCE_TYPE_STRING = "TABLE"; + private static final String TEST_RESOURCE_IDENTIFIER = "test-table"; + private static final String EMPTY_JSON = "{}"; + private static final String TEST_JSON = "{\"key\":\"value\"}"; + + // Dummy values for test initialization + private static final String DUMMY = "dummy"; + private static final long DUMMY_TIMESTAMP = 0L; + private static final PolarisEvent.ResourceType DUMMY_RESOURCE_TYPE = PolarisEvent.ResourceType.CATALOG; + + @Test + public void testFromResultSet() throws SQLException { + // Arrange + ResultSet mockResultSet = mock(ResultSet.class); + when(mockResultSet.getString(CATALOG_ID)).thenReturn(TEST_CATALOG_ID); + when(mockResultSet.getString(EVENT_ID)).thenReturn(TEST_EVENT_ID); + when(mockResultSet.getString(REQUEST_ID)).thenReturn(TEST_REQUEST_ID); + when(mockResultSet.getString(EVENT_TYPE)).thenReturn(TEST_EVENT_TYPE); + when(mockResultSet.getLong(TIMESTAMP_MS)).thenReturn(TEST_TIMESTAMP_MS); + when(mockResultSet.getString(ACTOR)).thenReturn(TEST_USER); + when(mockResultSet.getString(RESOURCE_TYPE)).thenReturn(TEST_RESOURCE_TYPE_STRING); + when(mockResultSet.getString(RESOURCE_IDENTIFIER)).thenReturn(TEST_RESOURCE_IDENTIFIER); + when(mockResultSet.getString(ADDITIONAL_PARAMETERS)).thenReturn(EMPTY_JSON); + + // Create a concrete implementation of ModelEvent for testing + ModelEvent modelEvent = ImmutableModelEvent.builder() + .catalogId(DUMMY) + .eventId(DUMMY) + .requestId(DUMMY) + .eventType(DUMMY) + .timestampMs(DUMMY_TIMESTAMP) + .principalName(DUMMY) + .resourceType(DUMMY_RESOURCE_TYPE) + .resourceIdentifier(DUMMY) + .additionalParameters(EMPTY_JSON) + .build(); + + // Act + PolarisEvent result = modelEvent.fromResultSet(mockResultSet); + + // Assert + assertEquals(TEST_CATALOG_ID, result.getCatalogId()); + assertEquals(TEST_EVENT_ID, result.getId()); + assertEquals(TEST_REQUEST_ID, result.getRequestId()); + assertEquals(TEST_EVENT_TYPE, result.getEventType()); + assertEquals(TEST_TIMESTAMP_MS, result.getTimestampMs()); + assertEquals(TEST_USER, result.getPrincipalName()); + assertEquals(TEST_RESOURCE_TYPE, result.getResourceType()); + assertEquals(TEST_RESOURCE_IDENTIFIER, result.getResourceIdentifier()); + assertEquals(EMPTY_JSON, result.getAdditionalParameters()); + } + + @Test + public void testToMapWithH2DatabaseType() { + // Arrange + ModelEvent modelEvent = ImmutableModelEvent.builder() + .catalogId(TEST_CATALOG_ID) + .eventId(TEST_EVENT_ID) + .requestId(TEST_REQUEST_ID) + .eventType(TEST_EVENT_TYPE) + .timestampMs(TEST_TIMESTAMP_MS) + .principalName(TEST_USER) + .resourceType(TEST_RESOURCE_TYPE) + .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) + .additionalParameters(TEST_JSON) + .build(); + + // Act + Map resultMap = modelEvent.toMap(DatabaseType.H2); + + // Assert + assertEquals(TEST_CATALOG_ID, resultMap.get(CATALOG_ID)); + assertEquals(TEST_EVENT_ID, resultMap.get(EVENT_ID)); + assertEquals(TEST_REQUEST_ID, resultMap.get(REQUEST_ID)); + assertEquals(TEST_EVENT_TYPE, resultMap.get(EVENT_TYPE)); + assertEquals(TEST_TIMESTAMP_MS, resultMap.get(TIMESTAMP_MS)); + assertEquals(TEST_USER, resultMap.get(PRINCIPAL_NAME)); + assertEquals(TEST_RESOURCE_TYPE_STRING, resultMap.get(RESOURCE_TYPE)); + assertEquals(TEST_RESOURCE_IDENTIFIER, resultMap.get(RESOURCE_IDENTIFIER)); + assertEquals(TEST_JSON, resultMap.get(ADDITIONAL_PARAMETERS)); + } + + @Test + public void testToMapWithPostgresType() { + // Arrange + ModelEvent modelEvent = ImmutableModelEvent.builder() + .catalogId(TEST_CATALOG_ID) + .eventId(TEST_EVENT_ID) + .requestId(TEST_REQUEST_ID) + .eventType(TEST_EVENT_TYPE) + .timestampMs(TEST_TIMESTAMP_MS) + .principalName(TEST_USER) + .resourceType(TEST_RESOURCE_TYPE) + .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) + .additionalParameters(TEST_JSON) + .build(); + + // Act + Map resultMap = modelEvent.toMap(DatabaseType.POSTGRES); + + // Assert + assertEquals(TEST_CATALOG_ID, resultMap.get(CATALOG_ID)); + assertEquals(TEST_EVENT_ID, resultMap.get(EVENT_ID)); + assertEquals(TEST_REQUEST_ID, resultMap.get(REQUEST_ID)); + assertEquals(TEST_EVENT_TYPE, resultMap.get(EVENT_TYPE)); + assertEquals(TEST_TIMESTAMP_MS, resultMap.get(TIMESTAMP_MS)); + assertEquals(TEST_USER, resultMap.get(PRINCIPAL_NAME)); + assertEquals(TEST_RESOURCE_TYPE_STRING, resultMap.get(RESOURCE_TYPE)); + assertEquals(TEST_RESOURCE_IDENTIFIER, resultMap.get(RESOURCE_IDENTIFIER)); + + // For PostgreSQL, the additional parameters should be a PGobject of type "jsonb" + PGobject pgObject = (PGobject) resultMap.get(ADDITIONAL_PARAMETERS); + assertEquals("jsonb", pgObject.getType()); + assertEquals(TEST_JSON, pgObject.getValue()); + } + + @Test + public void testFromEventWithNullInput() { + // Act + ModelEvent result = ModelEvent.fromEvent(null); + + // Assert + assertNull(result); + } + + @Test + public void testFromEvent() { + // Arrange + PolarisEvent polarisEvent = new PolarisEvent( + TEST_CATALOG_ID, + TEST_EVENT_ID, + TEST_REQUEST_ID, + TEST_EVENT_TYPE, + TEST_TIMESTAMP_MS, + TEST_USER, + TEST_RESOURCE_TYPE, + TEST_RESOURCE_IDENTIFIER); + polarisEvent.setAdditionalParameters(TEST_JSON); + + // Act + ModelEvent result = ModelEvent.fromEvent(polarisEvent); + + // Assert + assertEquals(TEST_CATALOG_ID, result.getCatalogId()); + assertEquals(TEST_EVENT_ID, result.getEventId()); + assertEquals(TEST_REQUEST_ID, result.getRequestId()); + assertEquals(TEST_EVENT_TYPE, result.getEventType()); + assertEquals(TEST_TIMESTAMP_MS, result.getTimestampMs()); + assertEquals(TEST_USER, result.getPrincipalName()); + assertEquals(TEST_RESOURCE_TYPE, result.getResourceType()); + assertEquals(TEST_RESOURCE_IDENTIFIER, result.getResourceIdentifier()); + assertEquals(TEST_JSON, result.getAdditionalParameters()); + } + + @Test + public void testToEventWithNullInput() { + // Act + PolarisEvent result = ModelEvent.toEvent(null); + + // Assert + assertNull(result); + } + + @Test + public void testToEvent() { + // Arrange + ModelEvent modelEvent = ImmutableModelEvent.builder() + .catalogId(TEST_CATALOG_ID) + .eventId(TEST_EVENT_ID) + .requestId(TEST_REQUEST_ID) + .eventType(TEST_EVENT_TYPE) + .timestampMs(TEST_TIMESTAMP_MS) + .principalName(TEST_USER) + .resourceType(TEST_RESOURCE_TYPE) + .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) + .additionalParameters(TEST_JSON) + .build(); + + // Act + PolarisEvent result = ModelEvent.toEvent(modelEvent); + + // Assert + assertEquals(TEST_CATALOG_ID, result.getCatalogId()); + assertEquals(TEST_EVENT_ID, result.getId()); + assertEquals(TEST_REQUEST_ID, result.getRequestId()); + assertEquals(TEST_EVENT_TYPE, result.getEventType()); + assertEquals(TEST_TIMESTAMP_MS, result.getTimestampMs()); + assertEquals(TEST_USER, result.getPrincipalName()); + assertEquals(TEST_RESOURCE_TYPE, result.getResourceType()); + assertEquals(TEST_RESOURCE_IDENTIFIER, result.getResourceIdentifier()); + assertEquals(TEST_JSON, result.getAdditionalParameters()); + } +} diff --git a/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java b/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java index cf2a0cca4c..90f930e176 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java @@ -21,6 +21,9 @@ import jakarta.annotation.Nonnull; import java.time.Clock; import java.time.ZoneId; +import java.util.UUID; + +import jakarta.ws.rs.container.ContainerRequestContext; import org.apache.polaris.core.config.PolarisConfigurationStore; import org.apache.polaris.core.config.RealmConfig; import org.apache.polaris.core.config.RealmConfigImpl; @@ -44,22 +47,34 @@ public class PolarisCallContext implements CallContext { private final Clock clock; + // A request ID to identify this REST request + private final String requestId; + private final RealmContext realmContext; private final RealmConfig realmConfig; + private static final String REQUEST_ID_KEY = "requestId"; + public PolarisCallContext( @Nonnull RealmContext realmContext, @Nonnull BasePersistence metaStore, @Nonnull PolarisDiagnostics diagServices, @Nonnull PolarisConfigurationStore configurationStore, - @Nonnull Clock clock) { + @Nonnull Clock clock, + ContainerRequestContext containerRequestContext) { this.realmContext = realmContext; this.metaStore = metaStore; this.diagServices = diagServices; this.configurationStore = configurationStore; this.clock = clock; this.realmConfig = new RealmConfigImpl(this.configurationStore, this.realmContext); + + String requestId = null; + if (containerRequestContext != null) { + requestId = (String) containerRequestContext.getProperty(REQUEST_ID_KEY); + } + this.requestId = requestId != null ? requestId : UUID.randomUUID().toString(); } public PolarisCallContext( @@ -71,7 +86,8 @@ public PolarisCallContext( metaStore, diagServices, new PolarisConfigurationStore() {}, - Clock.system(ZoneId.systemDefault())); + Clock.system(ZoneId.systemDefault()), + null); } public BasePersistence getMetaStore() { @@ -86,6 +102,10 @@ public Clock getClock() { return clock; } + public String getRequestId() { + return requestId; + } + @Override public RealmContext getRealmContext() { return realmContext; @@ -111,6 +131,6 @@ public PolarisCallContext copy() { String realmId = this.realmContext.getRealmIdentifier(); RealmContext realmContext = () -> realmId; return new PolarisCallContext( - realmContext, this.metaStore, this.diagServices, this.configurationStore, this.clock); + realmContext, this.metaStore, this.diagServices, this.configurationStore, this.clock, null); } } diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java new file mode 100644 index 0000000000..63779a26f1 --- /dev/null +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java @@ -0,0 +1,137 @@ +/* + * 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.polaris.core.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.Map; + +public class PolarisEvent { + public static final String EMPTY_MAP_STRING = "{}"; + + // to serialize/deserialize properties + private static final ObjectMapper MAPPER = new ObjectMapper(); + + // catalog id + private String catalogId; + + // event id + private String id; + + // id of the request that generated this event + private String requestId; + + // event type that was fired + private String eventType; + + // timestamp in epoch milliseconds of when this event was emitted + private long timestampMs; + + // polaris principal who took this action + private String principalName; + + // Enum that states the type of resource was being operated on + private ResourceType resourceType; + + // Which resource was operated on + private String resourceIdentifier; + + // Additional parameters that were not earlier recorded + private String additionalParameters; + + public String getCatalogId() { + return catalogId; + } + + public String getId() { + return id; + } + + public String getRequestId() { + return requestId; + } + + public String getEventType() { + return eventType; + } + + public long getTimestampMs() { + return timestampMs; + } + + public String getPrincipalName() { + return principalName; + } + + public ResourceType getResourceType() { + return resourceType; + } + + public String getResourceIdentifier() { + return resourceIdentifier; + } + + public String getAdditionalParameters() { + return additionalParameters != null ? additionalParameters : EMPTY_MAP_STRING; + } + + public PolarisEvent( + String catalogId, + String id, + String requestId, + String eventType, + long timestampMs, + String actor, + ResourceType resourceType, + String resourceIdentifier) { + this.catalogId = catalogId; + this.id = id; + this.requestId = requestId; + this.eventType = eventType; + this.timestampMs = timestampMs; + this.principalName = actor; + this.resourceType = resourceType; + this.resourceIdentifier = resourceIdentifier; + } + + @JsonIgnore + public void setAdditionalParameters(Map properties) { + try { + this.additionalParameters = properties == null ? null : MAPPER.writeValueAsString(properties); + } catch (JsonProcessingException ex) { + throw new IllegalStateException( + String.format("Failed to serialize json. properties %s", properties), ex); + } + } + + public void setAdditionalParameters(String additionalParameters) { + this.additionalParameters = additionalParameters; + } + + public enum ResourceType { + CATALOG, + NAMESPACE, + TABLE, + VIEW + } +} diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEventManager.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEventManager.java new file mode 100644 index 0000000000..2acae49cf3 --- /dev/null +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEventManager.java @@ -0,0 +1,34 @@ +/* + * 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.polaris.core.entity; + +import jakarta.annotation.Nonnull; +import org.apache.polaris.core.PolarisCallContext; +import org.apache.polaris.core.persistence.BasePersistence; + +import java.util.List; + +public interface PolarisEventManager { + @Nonnull + default void writeEvents(@Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { + BasePersistence ms = callCtx.getMetaStore(); + ms.writeEvents(polarisEvents); + } +} diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java index 08bda144ce..dcb7cb3e74 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java @@ -44,6 +44,7 @@ import org.apache.polaris.core.entity.PolarisEntityId; import org.apache.polaris.core.entity.PolarisEntitySubType; import org.apache.polaris.core.entity.PolarisEntityType; +import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.entity.PolarisGrantRecord; import org.apache.polaris.core.entity.PolarisPrincipalSecrets; import org.apache.polaris.core.entity.PolarisPrivilege; @@ -1970,4 +1971,13 @@ private List loadPoliciesFromMappingRecords( .collect(Collectors.toList()); return ms.lookupEntities(callCtx, policyEntityIds); } + + @Nonnull + @Override + public void writeEvents(@Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { + // get metastore we should be using + BasePersistence ms = callCtx.getMetaStore(); + + ms.writeEvents(polarisEvents); + } } diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/BasePersistence.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/BasePersistence.java index 26ccd8de39..1bfb02e112 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/BasePersistence.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/BasePersistence.java @@ -33,6 +33,7 @@ import org.apache.polaris.core.entity.PolarisEntityCore; import org.apache.polaris.core.entity.PolarisEntityId; import org.apache.polaris.core.entity.PolarisEntityType; +import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.entity.PolarisGrantRecord; import org.apache.polaris.core.persistence.pagination.Page; import org.apache.polaris.core.persistence.pagination.PageToken; @@ -135,6 +136,13 @@ void writeEntities( void writeToGrantRecords( @Nonnull PolarisCallContext callCtx, @Nonnull PolarisGrantRecord grantRec); + /** + * Write all events to the events table. This is an append-only operation. + * + * @param events events to persist + */ + void writeEvents(@Nonnull List events); + /** * Delete this entity from the meta store. * diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/PolarisMetaStoreManager.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/PolarisMetaStoreManager.java index b2fec2ddd8..d3356a28fa 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/PolarisMetaStoreManager.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/PolarisMetaStoreManager.java @@ -33,6 +33,7 @@ import org.apache.polaris.core.entity.PolarisEntityId; import org.apache.polaris.core.entity.PolarisEntitySubType; import org.apache.polaris.core.entity.PolarisEntityType; +import org.apache.polaris.core.entity.PolarisEventManager; import org.apache.polaris.core.persistence.dao.entity.BaseResult; import org.apache.polaris.core.persistence.dao.entity.ChangeTrackingResult; import org.apache.polaris.core.persistence.dao.entity.CreateCatalogResult; @@ -56,7 +57,8 @@ public interface PolarisMetaStoreManager extends PolarisSecretsManager, PolarisGrantManager, PolarisCredentialVendor, - PolarisPolicyMappingManager { + PolarisPolicyMappingManager, + PolarisEventManager { /** * Bootstrap the Polaris service, creating the root catalog, root principal, and associated diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/TransactionWorkspaceMetaStoreManager.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/TransactionWorkspaceMetaStoreManager.java index a7138c9fe2..1218e518e8 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/TransactionWorkspaceMetaStoreManager.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/TransactionWorkspaceMetaStoreManager.java @@ -34,6 +34,7 @@ import org.apache.polaris.core.entity.PolarisEntityId; import org.apache.polaris.core.entity.PolarisEntitySubType; import org.apache.polaris.core.entity.PolarisEntityType; +import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.entity.PolarisPrivilege; import org.apache.polaris.core.persistence.dao.entity.BaseResult; import org.apache.polaris.core.persistence.dao.entity.ChangeTrackingResult; @@ -444,4 +445,12 @@ Optional> hasOverlappingSiblings( .fail("illegal_method_in_transaction_workspace", "loadPoliciesOnEntityByType"); return null; } + + @Nonnull + @Override + public void writeEvents(@Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { + callCtx + .getDiagServices() + .fail("illegal_method_in_transaction_workspace", "writeEvents"); + } } diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/transactional/AbstractTransactionalPersistence.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/transactional/AbstractTransactionalPersistence.java index f08b85e122..1448d1cb18 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/transactional/AbstractTransactionalPersistence.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/transactional/AbstractTransactionalPersistence.java @@ -32,6 +32,7 @@ import org.apache.polaris.core.entity.PolarisEntityCore; import org.apache.polaris.core.entity.PolarisEntityId; import org.apache.polaris.core.entity.PolarisEntityType; +import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.entity.PolarisGrantRecord; import org.apache.polaris.core.entity.PolarisPrincipalSecrets; import org.apache.polaris.core.persistence.EntityAlreadyExistsException; @@ -261,6 +262,11 @@ public void writeToGrantRecords( runActionInTransaction(callCtx, () -> this.writeToGrantRecordsInCurrentTxn(callCtx, grantRec)); } + @Override + public void writeEvents(@Nonnull List events) { + throw new UnsupportedOperationException("Not implemented for transactional persistence."); + } + /** {@inheritDoc} */ @Override public void deleteEntity(@Nonnull PolarisCallContext callCtx, @Nonnull PolarisBaseEntity entity) { diff --git a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java index f89615cf10..7d2b2dd0ba 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java @@ -41,7 +41,8 @@ public PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { new TreeMapTransactionalPersistenceImpl(store, Mockito.mock(), RANDOM_SECRETS), diagServices, new PolarisConfigurationStore() {}, - timeSource.withZone(ZoneId.systemDefault())); + timeSource.withZone(ZoneId.systemDefault()), + null); return new PolarisTestMetaStoreManager(new AtomicOperationMetaStoreManager(), callCtx); } diff --git a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java index 49a2bfcc00..8e390748b0 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java @@ -41,7 +41,8 @@ public PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { new TreeMapTransactionalPersistenceImpl(store, Mockito.mock(), RANDOM_SECRETS), diagServices, new PolarisConfigurationStore() {}, - timeSource.withZone(ZoneId.systemDefault())); + timeSource.withZone(ZoneId.systemDefault()), + null); return new PolarisTestMetaStoreManager(new TransactionalMetaStoreManagerImpl(), callCtx); } diff --git a/polaris-core/src/test/java/org/apache/polaris/core/storage/InMemoryStorageIntegrationTest.java b/polaris-core/src/test/java/org/apache/polaris/core/storage/InMemoryStorageIntegrationTest.java index d0bf9de8a9..3e847b434a 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/storage/InMemoryStorageIntegrationTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/storage/InMemoryStorageIntegrationTest.java @@ -111,7 +111,8 @@ public void testValidateAccessToLocationsWithWildcard(String s3Scheme) { return (T) config.get(configName); } }, - Clock.systemUTC()); + Clock.systemUTC(), + null); CallContext.setCurrentContext(polarisCallContext); Map> result = storage.validateAccessToLocations( diff --git a/runtime/admin/src/main/java/org/apache/polaris/admintool/BootstrapCommand.java b/runtime/admin/src/main/java/org/apache/polaris/admintool/BootstrapCommand.java index 2d75b4c17d..c1a772409b 100644 --- a/runtime/admin/src/main/java/org/apache/polaris/admintool/BootstrapCommand.java +++ b/runtime/admin/src/main/java/org/apache/polaris/admintool/BootstrapCommand.java @@ -83,7 +83,7 @@ static class SchemaInputOptions { @CommandLine.Option( names = {"-v", "--schema-version"}, paramLabel = "", - description = "The version of the schema to load in [1, 2, LATEST].") + description = "The version of the schema to load in [1, 2, 3, LATEST].") Integer schemaVersion; @CommandLine.Option( diff --git a/runtime/defaults/src/main/resources/application.properties b/runtime/defaults/src/main/resources/application.properties index 7e9fbadac3..71fe2798d1 100644 --- a/runtime/defaults/src/main/resources/application.properties +++ b/runtime/defaults/src/main/resources/application.properties @@ -120,12 +120,17 @@ polaris.features."SUPPORTED_EXTERNAL_CATALOG_AUTHENTICATION_TYPES"=["OAUTH", "BE # polaris.persistence.type=eclipse-link # polaris.persistence.type=in-memory-atomic polaris.persistence.type=in-memory +# polaris.persistence.type=relational-jdbc polaris.secrets-manager.type=in-memory polaris.file-io.type=default polaris.event-listener.type=no-op +# polaris.event-listener.type=persistence-in-memory-buffer +# polaris.event-listener.buffer-time=5000ms +# polaris.event-listener.max-buffer-size=5 + polaris.log.request-id-header-name=Polaris-Request-Id # polaris.log.mdc.aid=polaris diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/ProductionReadinessChecks.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/ProductionReadinessChecks.java index 4205ef4cfe..78ffaf1e98 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/ProductionReadinessChecks.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/ProductionReadinessChecks.java @@ -42,8 +42,8 @@ import org.apache.polaris.service.context.DefaultRealmContextResolver; import org.apache.polaris.service.context.RealmContextResolver; import org.apache.polaris.service.context.TestRealmContextResolver; -import org.apache.polaris.service.events.PolarisEventListener; -import org.apache.polaris.service.events.TestPolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; +import org.apache.polaris.service.events.listeners.TestPolarisEventListener; import org.apache.polaris.service.persistence.InMemoryPolarisMetaStoreManagerFactory; import org.apache.polaris.service.quarkus.auth.QuarkusAuthenticationConfiguration; import org.eclipse.microprofile.config.Config; @@ -202,7 +202,7 @@ public ProductionReadinessCheck checkPolarisEventListener( PolarisEventListener polarisEventListener) { if (polarisEventListener instanceof TestPolarisEventListener) { return ProductionReadinessCheck.of( - Error.of("TestPolarisEventListener is intended for tests only.", "polaris.events.type")); + Error.of("TestPolarisEventListener is intended for tests only.", "polaris.event-listener.type")); } return ProductionReadinessCheck.OK; } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java index a9f9b2e77f..2a025e0b25 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java @@ -63,7 +63,7 @@ import org.apache.polaris.service.config.RealmEntityManagerFactory; import org.apache.polaris.service.context.RealmContextConfiguration; import org.apache.polaris.service.context.RealmContextResolver; -import org.apache.polaris.service.events.PolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.quarkus.auth.QuarkusAuthenticationConfiguration; import org.apache.polaris.service.quarkus.auth.QuarkusAuthenticationRealmConfiguration; import org.apache.polaris.service.quarkus.auth.external.tenant.OidcTenantResolver; @@ -131,16 +131,17 @@ public CallContext polarisCallContext( PolarisDiagnostics diagServices, PolarisConfigurationStore configurationStore, MetaStoreManagerFactory metaStoreManagerFactory, - Clock clock) { + Clock clock, + ContainerRequestContext containerRequestContext) { BasePersistence metaStoreSession = metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(); return new PolarisCallContext( - realmContext, metaStoreSession, diagServices, configurationStore, clock); + realmContext, metaStoreSession, diagServices, configurationStore, clock, containerRequestContext); } @Produces @RequestScoped - public RealmConfig realmContext(CallContext callContext) { + public RealmConfig realmConfig(CallContext callContext) { return callContext.getRealmConfig(); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java index 8921c726c6..9a6adc2f83 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java @@ -21,12 +21,23 @@ import io.quarkus.runtime.annotations.StaticInitSafe; import io.smallrye.config.ConfigMapping; +import java.time.Duration; +import java.util.Optional; +import org.apache.polaris.service.events.EventListenerConfiguration; +import org.apache.polaris.service.events.listeners.PolarisEventListener; + @StaticInitSafe @ConfigMapping(prefix = "polaris.event-listener") -public interface QuarkusPolarisEventListenerConfiguration { +public interface QuarkusPolarisEventListenerConfiguration extends EventListenerConfiguration { /** - * The type of the event listener to use. Must be a registered {@link - * org.apache.polaris.service.events.PolarisEventListener} identifier. + * The type of the event listener to use. Must be a registered {@link PolarisEventListener} + * identifier. */ String type(); + + @Override + Optional bufferTime(); + + @Override + Optional maxBufferSize(); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/task/QuarkusTaskExecutorImpl.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/task/QuarkusTaskExecutorImpl.java index 3e16edb5a6..80576e6897 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/task/QuarkusTaskExecutorImpl.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/task/QuarkusTaskExecutorImpl.java @@ -29,7 +29,7 @@ import java.util.concurrent.ExecutorService; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.persistence.MetaStoreManagerFactory; -import org.apache.polaris.service.events.PolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.quarkus.tracing.QuarkusTracingFilter; import org.apache.polaris.service.task.TaskExecutorImpl; import org.apache.polaris.service.task.TaskFileIOSupplier; diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java index 319a0e354c..1c6d4fa06b 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java @@ -85,7 +85,8 @@ public void setup() { .get(), fakeServices.polarisDiagnostics(), fakeServices.configurationStore(), - Mockito.mock(Clock.class)); + Mockito.mock(Clock.class), + null); CallContext.setCurrentContext(polarisCallContext); services = TestServices.builder() diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/PolarisAuthzTestBase.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/PolarisAuthzTestBase.java index 536e994b9d..6b2f27b527 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/PolarisAuthzTestBase.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/PolarisAuthzTestBase.java @@ -29,6 +29,7 @@ import jakarta.enterprise.context.RequestScoped; import jakarta.enterprise.inject.Alternative; import jakarta.inject.Inject; +import jakarta.ws.rs.container.ContainerRequestContext; import jakarta.ws.rs.core.SecurityContext; import java.io.IOException; import java.time.Clock; @@ -88,7 +89,7 @@ import org.apache.polaris.service.config.ReservedProperties; import org.apache.polaris.service.context.catalog.CallContextCatalogFactory; import org.apache.polaris.service.context.catalog.PolarisCallContextCatalogFactory; -import org.apache.polaris.service.events.PolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl; import org.apache.polaris.service.task.TaskExecutor; import org.apache.polaris.service.types.PolicyIdentifier; @@ -232,6 +233,9 @@ public static void setUpMocks() { public void before(TestInfo testInfo) { RealmContext realmContext = testInfo::getDisplayName; QuarkusMock.installMockForType(realmContext, RealmContext.class); + ContainerRequestContext containerRequestContext = Mockito.mock(ContainerRequestContext.class); + Mockito.when(containerRequestContext.getProperty(Mockito.anyString())).thenReturn("request-id-1"); + QuarkusMock.installMockForType(containerRequestContext, ContainerRequestContext.class); metaStoreManager = managerFactory.getOrCreateMetaStoreManager(realmContext); userSecretsManager = userSecretsManagerFactory.getOrCreateUserSecretsManager(realmContext); @@ -243,7 +247,8 @@ public void before(TestInfo testInfo) { managerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - clock); + clock, + null); this.entityManager = realmEntityManagerFactory.getOrCreateEntityManager(realmContext); callContext = polarisContext; diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTRSAKeyPairTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTRSAKeyPairTest.java index 46d2950b91..b633050ef8 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTRSAKeyPairTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTRSAKeyPairTest.java @@ -61,7 +61,7 @@ public void testSuccessfulTokenGeneration() throws Exception { final String scope = "PRINCIPAL_ROLE:TEST"; PolarisCallContext polarisCallContext = - new PolarisCallContext(null, null, null, configurationStore, null); + new PolarisCallContext(null, null, null, configurationStore, null, null); PolarisMetaStoreManager metastoreManager = Mockito.mock(PolarisMetaStoreManager.class); String mainSecret = "client-secret"; PolarisPrincipalSecrets principalSecrets = diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java index 6b51f6eda3..ff8a734584 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java @@ -45,7 +45,7 @@ public class JWTSymmetricKeyGeneratorTest { /** Sanity test to verify that we can generate a token */ @Test public void testJWTSymmetricKeyGenerator() { - PolarisCallContext polarisCallContext = new PolarisCallContext(null, null, null, null, null); + PolarisCallContext polarisCallContext = new PolarisCallContext(null, null, null, null, null, null); PolarisMetaStoreManager metastoreManager = Mockito.mock(PolarisMetaStoreManager.class); String mainSecret = "test_secret"; String clientId = "test_client_id"; diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogHandlerAuthzTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogHandlerAuthzTest.java index c6d848636d..6001f42ebd 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogHandlerAuthzTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogHandlerAuthzTest.java @@ -104,7 +104,8 @@ private IcebergCatalogHandler newWrapper( catalogName, polarisAuthorizer, reservedProperties, - catalogHandlerUtils); + catalogHandlerUtils, + polarisEventListener); } /** @@ -245,7 +246,8 @@ public void testInsufficientPermissionsPriorToSecretRotation() { CATALOG_NAME, polarisAuthorizer, reservedProperties, - catalogHandlerUtils); + catalogHandlerUtils, + polarisEventListener); // a variety of actions are all disallowed because the principal's credentials must be rotated doTestInsufficientPrivileges( @@ -280,7 +282,8 @@ public void testInsufficientPermissionsPriorToSecretRotation() { CATALOG_NAME, polarisAuthorizer, reservedProperties, - catalogHandlerUtils); + catalogHandlerUtils, + polarisEventListener); doTestSufficientPrivilegeSets( List.of(Set.of(PolarisPrivilege.NAMESPACE_LIST)), diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java index 2ef683b5d3..6bcd23bd84 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java @@ -140,8 +140,8 @@ import org.apache.polaris.service.events.AfterTableRefreshedEvent; import org.apache.polaris.service.events.BeforeTableCommitedEvent; import org.apache.polaris.service.events.BeforeTableRefreshedEvent; -import org.apache.polaris.service.events.PolarisEventListener; -import org.apache.polaris.service.events.TestPolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; +import org.apache.polaris.service.events.listeners.TestPolarisEventListener; import org.apache.polaris.service.exception.FakeAzureHttpResponse; import org.apache.polaris.service.exception.IcebergExceptionMapper; import org.apache.polaris.service.quarkus.config.QuarkusReservedProperties; @@ -284,7 +284,8 @@ public void before(TestInfo testInfo) { metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - Clock.systemDefaultZone()); + Clock.systemDefaultZone(), + null); entityManager = new PolarisEntityManager( @@ -2317,7 +2318,7 @@ public void testEventsAreEmitted() { Assertions.assertThat(afterRefreshEvent.tableIdentifier()).isEqualTo(TestData.TABLE); var beforeTableEvent = testPolarisEventListener.getLatest(BeforeTableCommitedEvent.class); - Assertions.assertThat(beforeTableEvent.identifier()).isEqualTo(TestData.TABLE); + Assertions.assertThat(beforeTableEvent.tableIdentifier()).isEqualTo(TestData.TABLE); Assertions.assertThat(beforeTableEvent.base().properties().get(key)).isEqualTo(valOld); Assertions.assertThat(beforeTableEvent.metadata().properties().get(key)).isEqualTo(valNew); diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogViewTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogViewTest.java index 4e516d015c..014b155e1e 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogViewTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogViewTest.java @@ -73,8 +73,8 @@ import org.apache.polaris.service.events.AfterViewRefreshedEvent; import org.apache.polaris.service.events.BeforeViewCommitedEvent; import org.apache.polaris.service.events.BeforeViewRefreshedEvent; -import org.apache.polaris.service.events.PolarisEventListener; -import org.apache.polaris.service.events.TestPolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; +import org.apache.polaris.service.events.listeners.TestPolarisEventListener; import org.apache.polaris.service.quarkus.test.TestData; import org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl; import org.assertj.core.api.Assertions; @@ -174,7 +174,8 @@ public void before(TestInfo testInfo) { metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - Clock.systemDefaultZone()); + Clock.systemDefaultZone(), + null); PolarisEntityManager entityManager = new PolarisEntityManager( diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolarisGenericTableCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolarisGenericTableCatalogTest.java index 7b8cfa3e99..6eac9f213e 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolarisGenericTableCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolarisGenericTableCatalogTest.java @@ -75,7 +75,7 @@ import org.apache.polaris.service.catalog.io.FileIOFactory; import org.apache.polaris.service.config.RealmEntityManagerFactory; import org.apache.polaris.service.config.ReservedProperties; -import org.apache.polaris.service.events.NoOpPolarisEventListener; +import org.apache.polaris.service.events.listeners.NoOpPolarisEventListener; import org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl; import org.apache.polaris.service.task.TaskExecutor; import org.assertj.core.api.Assertions; @@ -167,7 +167,8 @@ public void before(TestInfo testInfo) { metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - Clock.systemDefaultZone()); + Clock.systemDefaultZone(), + null); entityManager = new PolarisEntityManager( metaStoreManager, diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolicyCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolicyCatalogTest.java index cadfab7131..562ac3a3ff 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolicyCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolicyCatalogTest.java @@ -89,7 +89,7 @@ import org.apache.polaris.service.catalog.policy.PolicyCatalog; import org.apache.polaris.service.config.RealmEntityManagerFactory; import org.apache.polaris.service.config.ReservedProperties; -import org.apache.polaris.service.events.NoOpPolarisEventListener; +import org.apache.polaris.service.events.listeners.NoOpPolarisEventListener; import org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl; import org.apache.polaris.service.task.TaskExecutor; import org.apache.polaris.service.types.ApplicablePolicy; @@ -193,7 +193,8 @@ public void before(TestInfo testInfo) { metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - Clock.systemDefaultZone()); + Clock.systemDefaultZone(), + null); entityManager = new PolarisEntityManager( metaStoreManager, diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/ratelimiter/RateLimiterFilterTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/ratelimiter/RateLimiterFilterTest.java index f81ffe7744..06fd1642ce 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/ratelimiter/RateLimiterFilterTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/ratelimiter/RateLimiterFilterTest.java @@ -33,8 +33,8 @@ import java.util.Set; import java.util.function.Consumer; import org.apache.polaris.service.events.BeforeRequestRateLimitedEvent; -import org.apache.polaris.service.events.PolarisEventListener; -import org.apache.polaris.service.events.TestPolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; +import org.apache.polaris.service.events.listeners.TestPolarisEventListener; import org.apache.polaris.service.quarkus.ratelimiter.RateLimiterFilterTest.Profile; import org.apache.polaris.service.quarkus.test.PolarisIntegrationTestFixture; import org.apache.polaris.service.quarkus.test.PolarisIntegrationTestHelper; diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java index 9b03feee5b..4a683bb19e 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java @@ -106,7 +106,8 @@ void setup() { metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - Clock.systemDefaultZone()); + Clock.systemDefaultZone(), + null); } @Test diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java index e9268befb7..e3999b1578 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java @@ -118,7 +118,8 @@ private PolarisPrincipalSecrets fetchAdminSecrets() { metaStoreSession, helper.diagServices, helper.configurationStore, - helper.clock); + helper.clock, + null); try { PolarisMetaStoreManager metaStoreManager = helper.metaStoreManagerFactory.getOrCreateMetaStoreManager(realmContext); diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestHelper.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestHelper.java index 7149291543..e93c492f50 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestHelper.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestHelper.java @@ -22,6 +22,8 @@ import jakarta.inject.Inject; import jakarta.inject.Singleton; import java.time.Clock; + +import jakarta.ws.rs.container.ContainerRequestContext; import org.apache.polaris.core.PolarisDiagnostics; import org.apache.polaris.core.config.PolarisConfigurationStore; import org.apache.polaris.core.persistence.MetaStoreManagerFactory; diff --git a/service/common/build.gradle.kts b/service/common/build.gradle.kts index 2f5d958259..a55087689b 100644 --- a/service/common/build.gradle.kts +++ b/service/common/build.gradle.kts @@ -107,6 +107,7 @@ dependencies { testFixturesImplementation(libs.jakarta.enterprise.cdi.api) testFixturesImplementation(libs.jakarta.annotation.api) testFixturesImplementation(libs.jakarta.ws.rs.api) + testFixturesApi(libs.threeten.extra) testFixturesImplementation(platform(libs.quarkus.bom)) testFixturesImplementation("io.quarkus:quarkus-rest-client") diff --git a/service/common/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java b/service/common/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java index c9d36eb6b5..5680d5837a 100644 --- a/service/common/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java +++ b/service/common/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java @@ -79,6 +79,9 @@ import org.apache.polaris.service.admin.api.PolarisPrincipalsApiService; import org.apache.polaris.service.config.RealmEntityManagerFactory; import org.apache.polaris.service.config.ReservedProperties; +import org.apache.polaris.service.events.AfterCatalogCreatedEvent; +import org.apache.polaris.service.events.PolarisEvent; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.types.PolicyIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,6 +99,7 @@ public class PolarisServiceImpl private final UserSecretsManagerFactory userSecretsManagerFactory; private final CallContext callContext; private final ReservedProperties reservedProperties; + private final PolarisEventListener polarisEventListener; @Inject public PolarisServiceImpl( @@ -104,13 +108,15 @@ public PolarisServiceImpl( UserSecretsManagerFactory userSecretsManagerFactory, PolarisAuthorizer polarisAuthorizer, CallContext callContext, - ReservedProperties reservedProperties) { + ReservedProperties reservedProperties, + PolarisEventListener polarisEventListener) { this.entityManagerFactory = entityManagerFactory; this.metaStoreManagerFactory = metaStoreManagerFactory; this.userSecretsManagerFactory = userSecretsManagerFactory; this.polarisAuthorizer = polarisAuthorizer; this.callContext = callContext; this.reservedProperties = reservedProperties; + this.polarisEventListener = polarisEventListener; // FIXME: This is a hack to set the current context for downstream calls. CallContext.setCurrentContext(callContext); } @@ -149,6 +155,10 @@ public Response createCatalog( validateExternalCatalog(catalog); Catalog newCatalog = new CatalogEntity(adminService.createCatalog(request)).asCatalog(); LOGGER.info("Created new catalog {}", newCatalog); + polarisEventListener.onAfterCatalogCreated( + new AfterCatalogCreatedEvent(PolarisEvent.createEventId(), newCatalog.getName()), + callContext, + securityContext); return Response.status(Response.Status.CREATED).build(); } diff --git a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java index 1a8e769008..b37a83897c 100644 --- a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java +++ b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java @@ -138,7 +138,8 @@ import org.apache.polaris.service.events.BeforeTableRefreshedEvent; import org.apache.polaris.service.events.BeforeViewCommitedEvent; import org.apache.polaris.service.events.BeforeViewRefreshedEvent; -import org.apache.polaris.service.events.PolarisEventListener; +import org.apache.polaris.service.events.PolarisEvent; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.task.TaskExecutor; import org.apache.polaris.service.types.NotificationRequest; import org.apache.polaris.service.types.NotificationType; @@ -1427,7 +1428,10 @@ public void doRefresh() { if (latestLocation == null) { disableRefresh(); } else { - polarisEventListener.onBeforeTableRefreshed(new BeforeTableRefreshedEvent(tableIdentifier)); + polarisEventListener.onBeforeTableRefreshed( + new BeforeTableRefreshedEvent(PolarisEvent.createEventId(), tableIdentifier), + callContext, + securityContext); refreshFromMetadataLocation( latestLocation, SHOULD_RETRY_REFRESH_PREDICATE, @@ -1447,13 +1451,19 @@ public void doRefresh() { Set.of(PolarisStorageActions.READ, PolarisStorageActions.LIST)); return TableMetadataParser.read(fileIO, metadataLocation); }); - polarisEventListener.onAfterTableRefreshed(new AfterTableRefreshedEvent(tableIdentifier)); + polarisEventListener.onAfterTableRefreshed( + new AfterTableRefreshedEvent(PolarisEvent.createEventId(), tableIdentifier), + callContext, + securityContext); } } public void doCommit(TableMetadata base, TableMetadata metadata) { polarisEventListener.onBeforeTableCommited( - new BeforeTableCommitedEvent(tableIdentifier, base, metadata)); + new BeforeTableCommitedEvent( + PolarisEvent.createEventId(), tableIdentifier, base, metadata), + callContext, + securityContext); LOGGER.debug( "doCommit for table {} with base {}, metadata {}", tableIdentifier, base, metadata); @@ -1611,7 +1621,10 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { } polarisEventListener.onAfterTableCommited( - new AfterTableCommitedEvent(tableIdentifier, base, metadata)); + new AfterTableCommitedEvent( + PolarisEvent.createEventId(), tableIdentifier, base, metadata), + callContext, + securityContext); } @Override @@ -1802,7 +1815,10 @@ public void doRefresh() { if (latestLocation == null) { disableRefresh(); } else { - polarisEventListener.onBeforeViewRefreshed(new BeforeViewRefreshedEvent(identifier)); + polarisEventListener.onBeforeViewRefreshed( + new BeforeViewRefreshedEvent(PolarisEvent.createEventId(), identifier), + callContext, + securityContext); refreshFromMetadataLocation( latestLocation, SHOULD_RETRY_REFRESH_PREDICATE, @@ -1824,13 +1840,18 @@ public void doRefresh() { return ViewMetadataParser.read(fileIO.newInputFile(metadataLocation)); }); - polarisEventListener.onAfterViewRefreshed(new AfterViewRefreshedEvent(identifier)); + polarisEventListener.onAfterViewRefreshed( + new AfterViewRefreshedEvent(PolarisEvent.createEventId(), identifier), + callContext, + securityContext); } } public void doCommit(ViewMetadata base, ViewMetadata metadata) { polarisEventListener.onBeforeViewCommited( - new BeforeViewCommitedEvent(identifier, base, metadata)); + new BeforeViewCommitedEvent(PolarisEvent.createEventId(), identifier, base, metadata), + callContext, + securityContext); // TODO: Maybe avoid writing metadata if there's definitely a transaction conflict LOGGER.debug("doCommit for view {} with base {}, metadata {}", identifier, base, metadata); @@ -1927,7 +1948,9 @@ public void doCommit(ViewMetadata base, ViewMetadata metadata) { } polarisEventListener.onAfterViewCommited( - new AfterViewCommitedEvent(identifier, base, metadata)); + new AfterViewCommitedEvent(PolarisEvent.createEventId(), identifier, base, metadata), + callContext, + securityContext); } protected String writeNewMetadataIfRequired(ViewMetadata metadata) { diff --git a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java index 7218da0f31..fd7e7a996c 100644 --- a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java +++ b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java @@ -78,6 +78,7 @@ import org.apache.polaris.service.catalog.common.CatalogAdapter; import org.apache.polaris.service.config.ReservedProperties; import org.apache.polaris.service.context.catalog.CallContextCatalogFactory; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.http.IcebergHttpUtil; import org.apache.polaris.service.http.IfNoneMatch; import org.apache.polaris.service.types.CommitTableRequest; @@ -142,6 +143,7 @@ public class IcebergCatalogAdapter private final CatalogPrefixParser prefixParser; private final ReservedProperties reservedProperties; private final CatalogHandlerUtils catalogHandlerUtils; + private final PolarisEventListener polarisEventListener; @Inject public IcebergCatalogAdapter( @@ -154,7 +156,8 @@ public IcebergCatalogAdapter( PolarisAuthorizer polarisAuthorizer, CatalogPrefixParser prefixParser, ReservedProperties reservedProperties, - CatalogHandlerUtils catalogHandlerUtils) { + CatalogHandlerUtils catalogHandlerUtils, + PolarisEventListener polarisEventListener) { this.realmContext = realmContext; this.callContext = callContext; this.catalogFactory = catalogFactory; @@ -165,6 +168,7 @@ public IcebergCatalogAdapter( this.prefixParser = prefixParser; this.reservedProperties = reservedProperties; this.catalogHandlerUtils = catalogHandlerUtils; + this.polarisEventListener = polarisEventListener; // FIXME: This is a hack to set the current context for downstream calls. CallContext.setCurrentContext(callContext); @@ -204,7 +208,8 @@ IcebergCatalogHandler newHandlerWrapper(SecurityContext securityContext, String catalogName, polarisAuthorizer, reservedProperties, - catalogHandlerUtils); + catalogHandlerUtils, + polarisEventListener); } @Override diff --git a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java index c0b0b3259c..49de74a382 100644 --- a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java +++ b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java @@ -98,6 +98,9 @@ import org.apache.polaris.service.catalog.common.CatalogHandler; import org.apache.polaris.service.config.ReservedProperties; import org.apache.polaris.service.context.catalog.CallContextCatalogFactory; +import org.apache.polaris.service.events.AfterTableCreatedEvent; +import org.apache.polaris.service.events.PolarisEvent; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.http.IcebergHttpUtil; import org.apache.polaris.service.http.IfNoneMatch; import org.apache.polaris.service.types.NotificationRequest; @@ -127,6 +130,7 @@ public class IcebergCatalogHandler extends CatalogHandler implements AutoCloseab private final CallContextCatalogFactory catalogFactory; private final ReservedProperties reservedProperties; private final CatalogHandlerUtils catalogHandlerUtils; + private final PolarisEventListener polarisEventListener; // Catalog instance will be initialized after authorizing resolver successfully resolves // the catalog entity. @@ -147,13 +151,15 @@ public IcebergCatalogHandler( String catalogName, PolarisAuthorizer authorizer, ReservedProperties reservedProperties, - CatalogHandlerUtils catalogHandlerUtils) { + CatalogHandlerUtils catalogHandlerUtils, + PolarisEventListener polarisEventListener) { super(callContext, entityManager, securityContext, catalogName, authorizer); this.metaStoreManager = metaStoreManager; this.userSecretsManager = userSecretsManager; this.catalogFactory = catalogFactory; this.reservedProperties = reservedProperties; this.catalogHandlerUtils = catalogHandlerUtils; + this.polarisEventListener = polarisEventListener; } /** @@ -390,8 +396,14 @@ public LoadTableResponse createTableDirect(Namespace namespace, CreateTableReque .withWriteOrder(request.writeOrder()) .setProperties(reservedProperties.removeReservedProperties(request.properties())) .build(); - return catalogHandlerUtils.createTable( - baseCatalog, namespace, requestWithoutReservedProperties); + LoadTableResponse resp = + catalogHandlerUtils.createTable(baseCatalog, namespace, requestWithoutReservedProperties); + polarisEventListener.onAfterTableCreated( + new AfterTableCreatedEvent( + PolarisEvent.createEventId(), catalogName, resp.tableMetadata(), identifier), + callContext, + securityContext); + return resp; } /** @@ -439,15 +451,22 @@ public LoadTableResponse createTableDirectWithWriteDelegation( if (table instanceof BaseTable baseTable) { TableMetadata tableMetadata = baseTable.operations().current(); - return buildLoadTableResponseWithDelegationCredentials( - tableIdentifier, - tableMetadata, - Set.of( - PolarisStorageActions.READ, - PolarisStorageActions.WRITE, - PolarisStorageActions.LIST), - SNAPSHOTS_ALL) - .build(); + LoadTableResponse resp = + buildLoadTableResponseWithDelegationCredentials( + tableIdentifier, + tableMetadata, + Set.of( + PolarisStorageActions.READ, + PolarisStorageActions.WRITE, + PolarisStorageActions.LIST), + SNAPSHOTS_ALL) + .build(); + polarisEventListener.onAfterTableCreated( + new AfterTableCreatedEvent( + PolarisEvent.createEventId(), catalogName, resp.tableMetadata(), tableIdentifier), + callContext, + securityContext); + return resp; } else if (table instanceof BaseMetadataTable) { // metadata tables are loaded on the client side, return NoSuchTableException for now throw new NoSuchTableException("Table does not exist: %s", tableIdentifier.toString()); diff --git a/service/common/src/main/java/org/apache/polaris/service/context/DefaultCallContextResolver.java b/service/common/src/main/java/org/apache/polaris/service/context/DefaultCallContextResolver.java index 62b113daea..6af2b00af0 100644 --- a/service/common/src/main/java/org/apache/polaris/service/context/DefaultCallContextResolver.java +++ b/service/common/src/main/java/org/apache/polaris/service/context/DefaultCallContextResolver.java @@ -67,6 +67,6 @@ public CallContext resolveCallContext( BasePersistence metaStoreSession = metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(); return new PolarisCallContext( - realmContext, metaStoreSession, diagnostics, configurationStore, clock); + realmContext, metaStoreSession, diagnostics, configurationStore, clock, null); } } diff --git a/service/common/src/main/java/org/apache/polaris/service/context/catalog/PolarisCallContextCatalogFactory.java b/service/common/src/main/java/org/apache/polaris/service/context/catalog/PolarisCallContextCatalogFactory.java index 60c505d42c..43a7fa117f 100644 --- a/service/common/src/main/java/org/apache/polaris/service/context/catalog/PolarisCallContextCatalogFactory.java +++ b/service/common/src/main/java/org/apache/polaris/service/context/catalog/PolarisCallContextCatalogFactory.java @@ -36,7 +36,7 @@ import org.apache.polaris.service.catalog.iceberg.IcebergCatalog; import org.apache.polaris.service.catalog.io.FileIOFactory; import org.apache.polaris.service.config.RealmEntityManagerFactory; -import org.apache.polaris.service.events.PolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.task.TaskExecutor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java new file mode 100644 index 0000000000..dd125048a0 --- /dev/null +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java @@ -0,0 +1,24 @@ +/* + * 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.polaris.service.events; + +/** Emitted Polaris creates a catalog. */ +public record AfterCatalogCreatedEvent(String eventId, String catalogName) + implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java index c952997df1..c1988d79d7 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java @@ -30,5 +30,5 @@ * @param metadata The new metadata. */ public record AfterTableCommitedEvent( - TableIdentifier identifier, TableMetadata base, TableMetadata metadata) + String eventId, TableIdentifier identifier, TableMetadata base, TableMetadata metadata) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java new file mode 100644 index 0000000000..f55ad5aac3 --- /dev/null +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java @@ -0,0 +1,28 @@ +/* + * 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.polaris.service.events; + +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.catalog.TableIdentifier; + +/** Emitted when Polaris creates a table. */ +public record AfterTableCreatedEvent( + String eventId, String catalogName, TableMetadata metadata, TableIdentifier identifier) + implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java index be38a8baaa..d243789596 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java @@ -23,6 +23,7 @@ /** * Emitted after Polaris refreshes its known version of a table's metadata by fetching the latest. * + * @param eventId The event unique identifier * @param tableIdentifier The identifier of the table that was refreshed. */ -public record AfterTableRefreshedEvent(TableIdentifier tableIdentifier) implements PolarisEvent {} +public record AfterTableRefreshedEvent(String eventId, TableIdentifier tableIdentifier) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java index 638ba84fbc..9ae922d983 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java @@ -24,11 +24,10 @@ * Emitted after an attempt of an async task, such as manifest file cleanup, finishes. * * @param taskEntityId The ID of the TaskEntity. - * @param callContext The CallContext the task is being executed under. * @param attempt The attempt number. Each retry of the task will have its own attempt number. The * initial (non-retried) attempt starts counting from 1. - * @param success Whether or not the attempt succeeded. + * @param success Whether the attempt succeeded. */ public record AfterTaskAttemptedEvent( - long taskEntityId, CallContext callContext, int attempt, boolean success) + String eventId, long taskEntityId, int attempt, boolean success) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java index eb2ca24149..072a406a32 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java @@ -30,4 +30,5 @@ * @param metadata The new metadata. */ public record AfterViewCommitedEvent( - TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) implements PolarisEvent {} + String eventId, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) + implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java index 249220ddd7..4a11272e3f 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java @@ -25,4 +25,4 @@ * * @param viewIdentifier The identifier of the view that was refreshed. */ -public record AfterViewRefreshedEvent(TableIdentifier viewIdentifier) implements PolarisEvent {} +public record AfterViewRefreshedEvent(String eventId, TableIdentifier viewIdentifier) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeRequestRateLimitedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeRequestRateLimitedEvent.java index 1d9780ebe7..d3df44f3f3 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeRequestRateLimitedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeRequestRateLimitedEvent.java @@ -24,5 +24,5 @@ * @param method The request's HTTP method * @param absolutePath The request's absolute path */ -public record BeforeRequestRateLimitedEvent(String method, String absolutePath) +public record BeforeRequestRateLimitedEvent(String eventId, String method, String absolutePath) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java index 2bcc49ab67..ed1981e3b0 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java @@ -26,10 +26,10 @@ * of this event relative to the validation checks we've performed, which means the commit may still * fail Polaris-side validation checks. * - * @param identifier The identifier. + * @param tableIdentifier The identifier. * @param base The old metadata. * @param metadata The new metadata. */ public record BeforeTableCommitedEvent( - TableIdentifier identifier, TableMetadata base, TableMetadata metadata) + String eventId, TableIdentifier tableIdentifier, TableMetadata base, TableMetadata metadata) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java index f319298f57..bf350ca87e 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java @@ -26,4 +26,4 @@ * * @param tableIdentifier The identifier of the table being refreshed. */ -public record BeforeTableRefreshedEvent(TableIdentifier tableIdentifier) implements PolarisEvent {} +public record BeforeTableRefreshedEvent(String eventId, TableIdentifier tableIdentifier) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java index a7fa7231e7..03a807ad4b 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java @@ -24,9 +24,8 @@ * Emitted before an attempt of an async task, such as manifest file cleanup, begins. * * @param taskEntityId The ID of the TaskEntity - * @param callContext The CallContext the task is being executed under. * @param attempt The attempt number. Each retry of the task will have its own attempt number. The * initial (non-retried) attempt starts counting from 1. */ -public record BeforeTaskAttemptedEvent(long taskEntityId, CallContext callContext, int attempt) +public record BeforeTaskAttemptedEvent(String eventId, long taskEntityId, int attempt) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java index 16e460d806..2fa8296c98 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java @@ -31,4 +31,4 @@ * @param metadata The new metadata. */ public record BeforeViewCommitedEvent( - TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) implements PolarisEvent {} + String eventId, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java index 6f58d2ca22..3ff290a629 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java @@ -26,4 +26,4 @@ * * @param viewIdentifier The identifier of the view being refreshed. */ -public record BeforeViewRefreshedEvent(TableIdentifier viewIdentifier) implements PolarisEvent {} +public record BeforeViewRefreshedEvent(String eventId, TableIdentifier viewIdentifier) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java b/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java new file mode 100644 index 0000000000..6e91f79e36 --- /dev/null +++ b/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.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.polaris.service.events; + +import java.time.Duration; +import java.util.Optional; + +/** + * Configuration for event listeners. + * + *

bufferTime() specifies the buffer time in milliseconds. + *

maxBufferSize() specifies the maximum number of cached entries. + */ +public interface EventListenerConfiguration { + /** + * @return the buffer time in milliseconds + */ + Optional bufferTime(); + + /** + * @return the maximum number of cached entries + */ + Optional maxBufferSize(); +} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/PolarisEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/PolarisEvent.java index 4922c02f4c..b5a84c4937 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/PolarisEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/PolarisEvent.java @@ -18,8 +18,14 @@ */ package org.apache.polaris.service.events; +import java.util.UUID; + /** - * Represents an event emitted by Polaris. Currently there's no common data across events so this is - * just a marker interface. * + * Represents an event emitted by Polaris. Currently, there's no common data across events so this + * is just a marker interface. */ -public interface PolarisEvent {} +public interface PolarisEvent { + static String createEventId() { + return UUID.randomUUID().toString(); + } +} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/PolarisEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/PolarisEventListener.java deleted file mode 100644 index 485766bb24..0000000000 --- a/service/common/src/main/java/org/apache/polaris/service/events/PolarisEventListener.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.polaris.service.events; - -/** - * Represents an event listener that can respond to notable moments during Polaris's execution. - * Event details are documented under the event objects themselves. - */ -public abstract class PolarisEventListener { - /** {@link BeforeRequestRateLimitedEvent} */ - public void onBeforeRequestRateLimited(BeforeRequestRateLimitedEvent event) {} - - /** {@link BeforeTableCommitedEvent} */ - public void onBeforeTableCommited(BeforeTableCommitedEvent event) {} - - /** {@link AfterTableCommitedEvent} */ - public void onAfterTableCommited(AfterTableCommitedEvent event) {} - - /** {@link BeforeViewCommitedEvent} */ - public void onBeforeViewCommited(BeforeViewCommitedEvent event) {} - - /** {@link AfterViewCommitedEvent} */ - public void onAfterViewCommited(AfterViewCommitedEvent event) {} - - /** {@link BeforeTableRefreshedEvent} */ - public void onBeforeTableRefreshed(BeforeTableRefreshedEvent event) {} - - /** {@link AfterTableRefreshedEvent} */ - public void onAfterTableRefreshed(AfterTableRefreshedEvent event) {} - - /** {@link BeforeViewRefreshedEvent} */ - public void onBeforeViewRefreshed(BeforeViewRefreshedEvent event) {} - - /** {@link AfterViewRefreshedEvent} */ - public void onAfterViewRefreshed(AfterViewRefreshedEvent event) {} - - /** {@link BeforeTaskAttemptedEvent} */ - public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event) {} - - /** {@link AfterTaskAttemptedEvent} */ - public void onAfterTaskAttempted(AfterTaskAttemptedEvent event) {} -} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/TestPolarisEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/TestPolarisEventListener.java deleted file mode 100644 index 2e2538e890..0000000000 --- a/service/common/src/main/java/org/apache/polaris/service/events/TestPolarisEventListener.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.polaris.service.events; - -import com.google.common.collect.Streams; -import io.smallrye.common.annotation.Identifier; -import jakarta.enterprise.context.ApplicationScoped; -import java.util.ArrayList; -import java.util.List; - -/** Event listener that stores all emitted events forever. Not recommended for use in production. */ -@ApplicationScoped -@Identifier("test") -public class TestPolarisEventListener extends PolarisEventListener { - private final List history = new ArrayList<>(); - - public T getLatest(Class type) { - return (T) - Streams.findLast(history.stream().filter(type::isInstance)).map(type::cast).orElseThrow(); - } - - @Override - public void onBeforeRequestRateLimited(BeforeRequestRateLimitedEvent event) { - history.add(event); - } - - @Override - public void onBeforeTableCommited(BeforeTableCommitedEvent event) { - history.add(event); - } - - @Override - public void onAfterTableCommited(AfterTableCommitedEvent event) { - history.add(event); - } - - @Override - public void onBeforeViewCommited(BeforeViewCommitedEvent event) { - history.add(event); - } - - @Override - public void onAfterViewCommited(AfterViewCommitedEvent event) { - history.add(event); - } - - @Override - public void onBeforeTableRefreshed(BeforeTableRefreshedEvent event) { - history.add(event); - } - - @Override - public void onAfterTableRefreshed(AfterTableRefreshedEvent event) { - history.add(event); - } - - @Override - public void onBeforeViewRefreshed(BeforeViewRefreshedEvent event) { - history.add(event); - } - - @Override - public void onAfterViewRefreshed(AfterViewRefreshedEvent event) { - history.add(event); - } - - @Override - public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event) { - history.add(event); - } - - @Override - public void onAfterTaskAttempted(AfterTaskAttemptedEvent event) { - history.add(event); - } -} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java new file mode 100644 index 0000000000..e028f86fd9 --- /dev/null +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java @@ -0,0 +1,144 @@ +/* + * 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.polaris.service.events.listeners; + +import com.google.common.annotations.VisibleForTesting; +import io.smallrye.common.annotation.Identifier; +import jakarta.annotation.PostConstruct; +import jakarta.annotation.PreDestroy; +import jakarta.enterprise.context.ApplicationScoped; +import jakarta.inject.Inject; +import java.time.Clock; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.polaris.core.PolarisCallContext; +import org.apache.polaris.core.context.CallContext; +import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.core.persistence.MetaStoreManagerFactory; +import org.apache.polaris.service.events.EventListenerConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Event listener that buffers in memory and then dumps to persistence. */ +@ApplicationScoped +@Identifier("persistence-in-memory-buffer") +public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersistenceEventListener { + private static final Logger LOGGER = + LoggerFactory.getLogger(InMemoryBufferPolarisPersistenceEventListener.class); + private final MetaStoreManagerFactory metaStoreManagerFactory; + private final Clock clock; + + private final ConcurrentHashMap> buffer = new ConcurrentHashMap<>(); + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + private final ConcurrentHashMap, Integer> futures = new ConcurrentHashMap<>(); + private final Duration timeToFlush; + private final int maxBufferSize; + + private record EventAndContext(PolarisEvent polarisEvent, PolarisCallContext callContext) {} + + @Inject + public InMemoryBufferPolarisPersistenceEventListener( + MetaStoreManagerFactory metaStoreManagerFactory, + Clock clock, + EventListenerConfiguration eventListenerConfiguration) { + this.metaStoreManagerFactory = metaStoreManagerFactory; + this.clock = clock; + this.timeToFlush = + eventListenerConfiguration.bufferTime().orElse(Duration.of(30, ChronoUnit.SECONDS)); + this.maxBufferSize = eventListenerConfiguration.maxBufferSize().orElse(5); // 5 events default + } + + @PostConstruct + void start() { + futures.put( + executor.scheduleAtFixedRate(this::runCleanup, 0, timeToFlush.toMillis(), TimeUnit.MILLISECONDS), 1); + } + + void runCleanup() { + for (String realmId : buffer.keySet()) { + try { + checkAndFlushBufferIfNecessary(realmId); + } catch (Exception e) { + LOGGER.debug("Buffer checking task failed for realm ({}): {}", realmId, e); + } + } + // Clean up futures + try { + futures.keySet().removeIf(future -> future.isCancelled() || future.isDone()); + } catch (Exception e) { + LOGGER.debug("Futures reaper task failed."); + } + } + + @PreDestroy + void shutdown() { + futures.keySet().forEach(future -> future.cancel(false)); + executor.shutdownNow(); + } + + @Override + void addToBuffer(PolarisEvent polarisEvent, CallContext callCtx) { + String realmId = callCtx.getRealmContext().getRealmIdentifier(); + + buffer.computeIfAbsent(realmId, k -> new ConcurrentLinkedQueue<>()).add(new EventAndContext(polarisEvent, callCtx.getPolarisCallContext().copy())); + futures.put(executor.submit(() -> checkAndFlushBufferIfNecessary(realmId)), 1); + } + + @VisibleForTesting + public void checkAndFlushBufferIfNecessary(String realmId) { + ConcurrentLinkedQueue queue = buffer.get(realmId); + if (queue == null || queue.isEmpty()) { + return; + } + + // Given that we are using a ConcurrentLinkedQueue, this should not lock any calls to `add` on the queue. + synchronized (queue) { + // Double-check inside synchronized block + if (queue.isEmpty()) { + return; + } + + EventAndContext head = queue.peek(); + if (head == null) { + return; + } + + Duration elapsed = Duration.ofMillis(clock.millis() - head.polarisEvent.getTimestampMs()); + + if (elapsed.compareTo(timeToFlush) > 0 || queue.size() >= maxBufferSize) { + // Atomically replace old queue with new queue + boolean replaced = buffer.replace(realmId, queue, new ConcurrentLinkedQueue<>()); + if (!replaced) { + // Another thread concurrently modified the buffer, so do not continue + return; + } + + metaStoreManagerFactory.getOrCreateMetaStoreManager(() -> realmId).writeEvents(head.callContext(), new ArrayList<>(queue.stream().map(EventAndContext::polarisEvent).toList())); + } + } + } +} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/NoOpPolarisEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/NoOpPolarisEventListener.java similarity index 95% rename from service/common/src/main/java/org/apache/polaris/service/events/NoOpPolarisEventListener.java rename to service/common/src/main/java/org/apache/polaris/service/events/listeners/NoOpPolarisEventListener.java index f31fbcef51..c02dfe4811 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/NoOpPolarisEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/NoOpPolarisEventListener.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.polaris.service.events; +package org.apache.polaris.service.events.listeners; import io.smallrye.common.annotation.Identifier; import jakarta.enterprise.context.ApplicationScoped; diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java new file mode 100644 index 0000000000..dacb8ea734 --- /dev/null +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java @@ -0,0 +1,92 @@ +/* + * 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.polaris.service.events.listeners; + +import jakarta.ws.rs.core.SecurityContext; +import org.apache.polaris.core.context.CallContext; +import org.apache.polaris.service.events.AfterCatalogCreatedEvent; +import org.apache.polaris.service.events.AfterTableCommitedEvent; +import org.apache.polaris.service.events.AfterTableCreatedEvent; +import org.apache.polaris.service.events.AfterTableRefreshedEvent; +import org.apache.polaris.service.events.AfterTaskAttemptedEvent; +import org.apache.polaris.service.events.AfterViewCommitedEvent; +import org.apache.polaris.service.events.AfterViewRefreshedEvent; +import org.apache.polaris.service.events.BeforeRequestRateLimitedEvent; +import org.apache.polaris.service.events.BeforeTableCommitedEvent; +import org.apache.polaris.service.events.BeforeTableRefreshedEvent; +import org.apache.polaris.service.events.BeforeTaskAttemptedEvent; +import org.apache.polaris.service.events.BeforeViewCommitedEvent; +import org.apache.polaris.service.events.BeforeViewRefreshedEvent; + +/** + * Represents an event listener that can respond to notable moments during Polaris's execution. + * Event details are documented under the event objects themselves. + */ +public abstract class PolarisEventListener { + + /** {@link BeforeRequestRateLimitedEvent} */ + public void onBeforeRequestRateLimited( + BeforeRequestRateLimitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link BeforeTableCommitedEvent} */ + public void onBeforeTableCommited( + BeforeTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link AfterTableCommitedEvent} */ + public void onAfterTableCommited( + AfterTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link BeforeViewCommitedEvent} */ + public void onBeforeViewCommited( + BeforeViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link AfterViewCommitedEvent} */ + public void onAfterViewCommited( + AfterViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link BeforeTableRefreshedEvent} */ + public void onBeforeTableRefreshed( + BeforeTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link AfterTableRefreshedEvent} */ + public void onAfterTableRefreshed( + AfterTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link BeforeViewRefreshedEvent} */ + public void onBeforeViewRefreshed( + BeforeViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link AfterViewRefreshedEvent} */ + public void onAfterViewRefreshed( + AfterViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link BeforeTaskAttemptedEvent} */ + public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event, CallContext callCtx) {} + + /** {@link AfterTaskAttemptedEvent} */ + public void onAfterTaskAttempted(AfterTaskAttemptedEvent event, CallContext callCtx) {} + + /** {@link AfterTableCreatedEvent} */ + public void onAfterTableCreated( + AfterTableCreatedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + /** {@link AfterCatalogCreatedEvent} */ + public void onAfterCatalogCreated( + AfterCatalogCreatedEvent event, CallContext callCtx, SecurityContext securityContext) {} +} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java new file mode 100644 index 0000000000..dda314bc36 --- /dev/null +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java @@ -0,0 +1,139 @@ +/* + * 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.polaris.service.events.listeners; + +import jakarta.ws.rs.core.SecurityContext; +import java.util.Map; +import org.apache.iceberg.TableMetadataParser; +import org.apache.polaris.core.context.CallContext; +import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.service.events.AfterCatalogCreatedEvent; +import org.apache.polaris.service.events.AfterTableCommitedEvent; +import org.apache.polaris.service.events.AfterTableCreatedEvent; +import org.apache.polaris.service.events.AfterTableRefreshedEvent; +import org.apache.polaris.service.events.AfterTaskAttemptedEvent; +import org.apache.polaris.service.events.AfterViewCommitedEvent; +import org.apache.polaris.service.events.AfterViewRefreshedEvent; +import org.apache.polaris.service.events.BeforeRequestRateLimitedEvent; +import org.apache.polaris.service.events.BeforeTableCommitedEvent; +import org.apache.polaris.service.events.BeforeTableRefreshedEvent; +import org.apache.polaris.service.events.BeforeTaskAttemptedEvent; +import org.apache.polaris.service.events.BeforeViewCommitedEvent; +import org.apache.polaris.service.events.BeforeViewRefreshedEvent; + +public abstract class PolarisPersistenceEventListener extends PolarisEventListener { + @Override + public final void onBeforeRequestRateLimited( + BeforeRequestRateLimitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onBeforeTableCommited( + BeforeTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onAfterTableCommited( + AfterTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onBeforeViewCommited( + BeforeViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onAfterViewCommited( + AfterViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onBeforeTableRefreshed( + BeforeTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onAfterTableRefreshed( + AfterTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onBeforeViewRefreshed( + BeforeViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onAfterViewRefreshed( + AfterViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + + @Override + public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event, CallContext callCtx) {} + + @Override + public void onAfterTaskAttempted(AfterTaskAttemptedEvent event, CallContext callCtx) {} + + @Override + public void onAfterTableCreated( + AfterTableCreatedEvent event, CallContext callCtx, SecurityContext securityContext) { + org.apache.polaris.core.entity.PolarisEvent polarisEvent = + new org.apache.polaris.core.entity.PolarisEvent( + event.catalogName(), + event.eventId(), + getRequestId(callCtx), + event.getClass().getSimpleName(), + getTimestamp(callCtx), + getUsername(securityContext), + PolarisEvent.ResourceType.TABLE, + event.identifier().toString()); + Map additionalParameters = + Map.of( + "table-uuid", + event.metadata().uuid(), + "metadata", + TableMetadataParser.toJson(event.metadata())); + polarisEvent.setAdditionalParameters(additionalParameters); + + addToBuffer(polarisEvent, callCtx); + } + + @Override + public void onAfterCatalogCreated( + AfterCatalogCreatedEvent event, CallContext callCtx, SecurityContext securityContext) { + org.apache.polaris.core.entity.PolarisEvent polarisEvent = + new PolarisEvent( + event.catalogName(), + event.eventId(), + getRequestId(callCtx), + event.getClass().getSimpleName(), + getTimestamp(callCtx), + getUsername(securityContext), + PolarisEvent.ResourceType.CATALOG, + event.catalogName()); + addToBuffer(polarisEvent, callCtx); + } + + private long getTimestamp(CallContext callCtx) { + return callCtx.getPolarisCallContext().getClock().millis(); + } + + private String getRequestId(CallContext callCtx) { + return callCtx.getPolarisCallContext().getRequestId(); + } + + private String getUsername(SecurityContext securityContext) { + return securityContext.getUserPrincipal() == null + ? null + : securityContext.getUserPrincipal().getName(); + } + + abstract void addToBuffer(org.apache.polaris.core.entity.PolarisEvent event, CallContext callCtx); +} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java new file mode 100644 index 0000000000..511f790ba9 --- /dev/null +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java @@ -0,0 +1,115 @@ +/* + * 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.polaris.service.events.listeners; + +import com.google.common.collect.Streams; +import io.smallrye.common.annotation.Identifier; +import jakarta.enterprise.context.ApplicationScoped; +import jakarta.ws.rs.core.SecurityContext; +import java.util.ArrayList; +import java.util.List; +import org.apache.polaris.core.context.CallContext; +import org.apache.polaris.service.events.AfterTableCommitedEvent; +import org.apache.polaris.service.events.AfterTableRefreshedEvent; +import org.apache.polaris.service.events.AfterTaskAttemptedEvent; +import org.apache.polaris.service.events.AfterViewCommitedEvent; +import org.apache.polaris.service.events.AfterViewRefreshedEvent; +import org.apache.polaris.service.events.BeforeRequestRateLimitedEvent; +import org.apache.polaris.service.events.BeforeTableCommitedEvent; +import org.apache.polaris.service.events.BeforeTableRefreshedEvent; +import org.apache.polaris.service.events.BeforeTaskAttemptedEvent; +import org.apache.polaris.service.events.BeforeViewCommitedEvent; +import org.apache.polaris.service.events.BeforeViewRefreshedEvent; +import org.apache.polaris.service.events.PolarisEvent; + +/** Event listener that stores all emitted events forever. Not recommended for use in production. */ +@ApplicationScoped +@Identifier("test") +public class TestPolarisEventListener extends PolarisEventListener { + private final List history = new ArrayList<>(); + + public T getLatest(Class type) { + return (T) + Streams.findLast(history.stream().filter(type::isInstance)).map(type::cast).orElseThrow(); + } + + @Override + public void onBeforeRequestRateLimited( + BeforeRequestRateLimitedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onBeforeTableCommited( + BeforeTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onAfterTableCommited( + AfterTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onBeforeViewCommited( + BeforeViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onAfterViewCommited( + AfterViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onBeforeTableRefreshed( + BeforeTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onAfterTableRefreshed( + AfterTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onBeforeViewRefreshed( + BeforeViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onAfterViewRefreshed( + AfterViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) { + history.add(event); + } + + @Override + public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event, CallContext callCtx) { + history.add(event); + } + + @Override + public void onAfterTaskAttempted(AfterTaskAttemptedEvent event, CallContext callCtx) { + history.add(event); + } +} diff --git a/service/common/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java b/service/common/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java index 7b0fda93fc..d13590f3bd 100644 --- a/service/common/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java +++ b/service/common/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java @@ -27,9 +27,11 @@ import jakarta.ws.rs.core.Response; import jakarta.ws.rs.ext.Provider; import java.io.IOException; +import org.apache.polaris.core.context.CallContext; import org.apache.polaris.service.config.PolarisFilterPriorities; import org.apache.polaris.service.events.BeforeRequestRateLimitedEvent; -import org.apache.polaris.service.events.PolarisEventListener; +import org.apache.polaris.service.events.PolarisEvent; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,7 +58,11 @@ public void filter(ContainerRequestContext ctx) throws IOException { if (!rateLimiter.canProceed()) { polarisEventListener.onBeforeRequestRateLimited( new BeforeRequestRateLimitedEvent( - ctx.getMethod(), ctx.getUriInfo().getAbsolutePath().toString())); + PolarisEvent.createEventId(), + ctx.getMethod(), + ctx.getUriInfo().getAbsolutePath().toString()), + CallContext.getCurrentContext(), + ctx.getSecurityContext()); ctx.abortWith(Response.status(Response.Status.TOO_MANY_REQUESTS).build()); LOGGER.atDebug().log("Rate limiting request"); } diff --git a/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java b/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java index c1c775bf42..09b2fd487e 100644 --- a/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java +++ b/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java @@ -35,7 +35,8 @@ import org.apache.polaris.core.persistence.PolarisMetaStoreManager; import org.apache.polaris.service.events.AfterTaskAttemptedEvent; import org.apache.polaris.service.events.BeforeTaskAttemptedEvent; -import org.apache.polaris.service.events.PolarisEventListener; +import org.apache.polaris.service.events.PolarisEvent; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,7 +119,8 @@ public void addTaskHandlerContext(long taskEntityId, CallContext callContext) { protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { polarisEventListener.onBeforeTaskAttempted( - new BeforeTaskAttemptedEvent(taskEntityId, ctx, attempt)); + new BeforeTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt), + ctx); boolean success = false; try { @@ -164,7 +166,9 @@ protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { } } finally { polarisEventListener.onAfterTaskAttempted( - new AfterTaskAttemptedEvent(taskEntityId, ctx, attempt, success)); + new AfterTaskAttemptedEvent( + PolarisEvent.createEventId(), taskEntityId, attempt, success), + ctx); } } } diff --git a/service/common/src/test/java/org/apache/polaris/service/admin/PolarisServiceImplTest.java b/service/common/src/test/java/org/apache/polaris/service/admin/PolarisServiceImplTest.java index a6d01ede47..01619daf2a 100644 --- a/service/common/src/test/java/org/apache/polaris/service/admin/PolarisServiceImplTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/admin/PolarisServiceImplTest.java @@ -40,6 +40,8 @@ import org.apache.polaris.core.secrets.UserSecretsManagerFactory; import org.apache.polaris.service.config.RealmEntityManagerFactory; import org.apache.polaris.service.config.ReservedProperties; +import org.apache.polaris.service.events.listeners.NoOpPolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -53,6 +55,7 @@ public class PolarisServiceImplTest { private CallContext callContext; private ReservedProperties reservedProperties; private RealmConfig realmConfig; + private PolarisEventListener polarisEventListener; private PolarisServiceImpl polarisService; @@ -65,6 +68,7 @@ void setUp() { callContext = Mockito.mock(CallContext.class); reservedProperties = Mockito.mock(ReservedProperties.class); realmConfig = Mockito.mock(RealmConfig.class); + polarisEventListener = new NoOpPolarisEventListener(); when(callContext.getRealmConfig()).thenReturn(realmConfig); when(realmConfig.getConfig(FeatureConfiguration.SUPPORTED_CATALOG_CONNECTION_TYPES)) @@ -80,7 +84,8 @@ void setUp() { userSecretsManagerFactory, polarisAuthorizer, callContext, - reservedProperties); + reservedProperties, + polarisEventListener); } @Test diff --git a/service/common/src/test/java/org/apache/polaris/service/catalog/io/FileIOFactoryTest.java b/service/common/src/test/java/org/apache/polaris/service/catalog/io/FileIOFactoryTest.java index 34a9531d23..1ccd6a0c09 100644 --- a/service/common/src/test/java/org/apache/polaris/service/catalog/io/FileIOFactoryTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/catalog/io/FileIOFactoryTest.java @@ -142,7 +142,8 @@ FileIO loadFileIOInternal( testServices.metaStoreManagerFactory().getOrCreateSessionSupplier(realmContext).get(), testServices.polarisDiagnostics(), testServices.configurationStore(), - Clock.systemUTC()); + Clock.systemUTC(), + null); } @AfterEach diff --git a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java new file mode 100644 index 0000000000..21c4d34f48 --- /dev/null +++ b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -0,0 +1,197 @@ +/* + * 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.polaris.service.events.listeners; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.time.Instant; +import java.time.ZoneOffset; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.polaris.core.PolarisCallContext; +import org.apache.polaris.core.context.CallContext; +import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.core.persistence.MetaStoreManagerFactory; +import org.apache.polaris.core.persistence.PolarisMetaStoreManager; +import org.apache.polaris.service.events.EventListenerConfiguration; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; +import org.threeten.extra.MutableClock; + +public class InMemoryBufferPolarisPersistenceEventListenerTest { + private InMemoryBufferPolarisPersistenceEventListener eventListener; + private PolarisMetaStoreManager polarisMetaStoreManager; + private MutableClock clock; + private CallContext callContext; + + private static final int CONFIG_MAX_BUFFER_SIZE = 5; + private static final Duration CONFIG_TIME_TO_FLUSH_IN_MS = Duration.ofMillis(500); + + @BeforeEach + public void setUp() { + callContext = Mockito.mock(CallContext.class); + PolarisCallContext polarisCallContext = Mockito.mock(PolarisCallContext.class); + when(callContext.getPolarisCallContext()).thenReturn(polarisCallContext); + when(polarisCallContext.copy()).thenReturn(polarisCallContext); + + MetaStoreManagerFactory metaStoreManagerFactory = Mockito.mock(MetaStoreManagerFactory.class); + polarisMetaStoreManager = Mockito.mock(PolarisMetaStoreManager.class); + when(metaStoreManagerFactory.getOrCreateMetaStoreManager(Mockito.any())) + .thenReturn(polarisMetaStoreManager); + + EventListenerConfiguration eventListenerConfiguration = + Mockito.mock(EventListenerConfiguration.class); + when(eventListenerConfiguration.maxBufferSize()) + .thenReturn(Optional.of(CONFIG_MAX_BUFFER_SIZE)); + when(eventListenerConfiguration.bufferTime()) + .thenReturn(Optional.of(CONFIG_TIME_TO_FLUSH_IN_MS)); + + clock = + MutableClock.of( + Instant.ofEpochSecond(0), ZoneOffset.UTC); // Use 0 Epoch Time to make it easier to test + + eventListener = + new InMemoryBufferPolarisPersistenceEventListener( + metaStoreManagerFactory, clock, eventListenerConfiguration); + } + + @Test + public void testAddToBufferFlushesAfterConfiguredTime() { + String realmId = "realm1"; + List eventsAddedToBuffer = addEventsWithoutTriggeringFlush(realmId); + + // Push clock forwards to flush the buffer + clock.add(CONFIG_TIME_TO_FLUSH_IN_MS.multipliedBy(2)); + eventListener.checkAndFlushBufferIfNecessary(realmId); + verify(polarisMetaStoreManager, times(1)).writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); + } + + @Test + public void testAddToBufferFlushesAfterMaxEvents() { + String realm1 = "realm1"; + List eventsAddedToBuffer = addEventsWithoutTriggeringFlush(realm1); + addEventsWithoutTriggeringFlush("realm2"); + + // Add the last event for realm1 and verify that it did trigger the flush + PolarisEvent triggeringEvent = createSampleEvent(); + RealmContext realmContext = () -> realm1; + when(callContext.getRealmContext()).thenReturn(realmContext); + eventListener.addToBuffer(triggeringEvent, callContext); + eventsAddedToBuffer.add(triggeringEvent); + + // Given the call to checkAndFlushBufferIfNecessary is async, the calling thread should not have blocked and nothing would've been done immediately + verify(polarisMetaStoreManager, times(0)).writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); + + // Calling checkAndFlushBufferIfNecessary manually to replicate the behavior of the executor service + eventListener.checkAndFlushBufferIfNecessary(realm1); + verify(polarisMetaStoreManager, times(0)).writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); + } + + @Test + public void testCheckAndFlushBufferIfNecessaryIsThreadSafe() throws Exception { + String realmId = "realm1"; + int threadCount = 10; + List threads = new ArrayList<>(); + ConcurrentLinkedQueue exceptions = new ConcurrentLinkedQueue<>(); + + // Pre-populate the buffer with events + List events = addEventsWithoutTriggeringFlush(realmId); + + // Push clock forwards to flush the buffer + clock.add(CONFIG_TIME_TO_FLUSH_IN_MS.multipliedBy(2)); + + // Each thread will call checkAndFlushBufferIfNecessary concurrently + for (int i = 0; i < threadCount; i++) { + Thread t = new Thread(() -> { + try { + eventListener.checkAndFlushBufferIfNecessary(realmId); + } catch (Exception e) { + exceptions.add(e); + } + }); + threads.add(t); + } + // Start all threads + threads.forEach(Thread::start); + // Wait for all threads to finish + for (Thread t : threads) { + t.join(); + } + // There should be no exceptions + if (!exceptions.isEmpty()) { + throw new AssertionError("Exceptions occurred in concurrent checkAndFlushBufferIfNecessary: ", exceptions.peek()); + } + // Only one flush should occur + verify(polarisMetaStoreManager, times(1)).writeEvents(eq(callContext.getPolarisCallContext()), eq(events)); + } + + private List addEventsWithoutTriggeringFlush(String realmId) { + List realmEvents = new ArrayList<>(); + for (int i = 0; i < CONFIG_MAX_BUFFER_SIZE - 1; i++) { + realmEvents.add(createSampleEvent()); + } + RealmContext realmContext = () -> realmId; + when(callContext.getRealmContext()).thenReturn(realmContext); + for (PolarisEvent realmEvent : realmEvents) { + eventListener.addToBuffer(realmEvent, callContext); + } + verify(polarisMetaStoreManager, times(0)).writeEvents(Mockito.any(), Mockito.any()); + return realmEvents; + } + + private PolarisEvent createSampleEvent() { + String catalogId = "test-catalog"; + String id = UUID.randomUUID().toString(); + String requestId = "test-request-id"; + String eventType = "TEST_EVENT"; + long timestampMs = 0; + String principalName = "test-user"; + PolarisEvent.ResourceType resourceType = PolarisEvent.ResourceType.TABLE; + String resourceIdentifier = "test-table"; + + PolarisEvent event = new PolarisEvent( + catalogId, + id, + requestId, + eventType, + timestampMs, + principalName, + resourceType, + resourceIdentifier); + + Map additionalParams = new HashMap<>(); + additionalParams.put("testKey", "testValue"); + event.setAdditionalParameters(additionalParams); + + return event; + } +} diff --git a/service/common/src/test/java/org/apache/polaris/service/task/TaskExecutorImplTest.java b/service/common/src/test/java/org/apache/polaris/service/task/TaskExecutorImplTest.java index c67f7f172a..9beac2beea 100644 --- a/service/common/src/test/java/org/apache/polaris/service/task/TaskExecutorImplTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/task/TaskExecutorImplTest.java @@ -28,7 +28,7 @@ import org.apache.polaris.service.TestServices; import org.apache.polaris.service.events.AfterTaskAttemptedEvent; import org.apache.polaris.service.events.BeforeTaskAttemptedEvent; -import org.apache.polaris.service.events.TestPolarisEventListener; +import org.apache.polaris.service.events.listeners.TestPolarisEventListener; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -81,7 +81,6 @@ public boolean handleTask(TaskEntity task, CallContext callContext) { var beforeTaskAttemptedEvent = testPolarisEventListener.getLatest(BeforeTaskAttemptedEvent.class); Assertions.assertEquals(taskEntity.getId(), beforeTaskAttemptedEvent.taskEntityId()); - Assertions.assertEquals(callContext, beforeTaskAttemptedEvent.callContext()); Assertions.assertEquals(attempt, beforeTaskAttemptedEvent.attempt()); return true; } @@ -90,8 +89,8 @@ public boolean handleTask(TaskEntity task, CallContext callContext) { executor.handleTask(taskEntity.getId(), polarisCallCtx, attempt); var afterAttemptTaskEvent = testPolarisEventListener.getLatest(AfterTaskAttemptedEvent.class); + Assertions.assertEquals(taskEntity.getId(), afterAttemptTaskEvent.taskEntityId()); - Assertions.assertEquals(polarisCallCtx, afterAttemptTaskEvent.callContext()); Assertions.assertEquals(attempt, afterAttemptTaskEvent.attempt()); Assertions.assertTrue(afterAttemptTaskEvent.success()); } diff --git a/service/common/src/testFixtures/java/org/apache/polaris/service/TestServices.java b/service/common/src/testFixtures/java/org/apache/polaris/service/TestServices.java index e8f1950d5d..28053ed7ff 100644 --- a/service/common/src/testFixtures/java/org/apache/polaris/service/TestServices.java +++ b/service/common/src/testFixtures/java/org/apache/polaris/service/TestServices.java @@ -60,8 +60,8 @@ import org.apache.polaris.service.config.ReservedProperties; import org.apache.polaris.service.context.catalog.CallContextCatalogFactory; import org.apache.polaris.service.context.catalog.PolarisCallContextCatalogFactory; -import org.apache.polaris.service.events.PolarisEventListener; -import org.apache.polaris.service.events.TestPolarisEventListener; +import org.apache.polaris.service.events.listeners.PolarisEventListener; +import org.apache.polaris.service.events.listeners.TestPolarisEventListener; import org.apache.polaris.service.persistence.InMemoryPolarisMetaStoreManagerFactory; import org.apache.polaris.service.secrets.UnsafeInMemorySecretsManagerFactory; import org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl; @@ -165,7 +165,8 @@ public TestServices build() { metaStoreSession, polarisDiagnostics, configurationStore, - Clock.systemUTC()); + Clock.systemUTC(), + null); PolarisEntityManager entityManager = realmEntityManagerFactory.getOrCreateEntityManager(realmContext); PolarisMetaStoreManager metaStoreManager = @@ -204,7 +205,8 @@ public TestServices build() { authorizer, new DefaultCatalogPrefixParser(), reservedProperties, - catalogHandlerUtils); + catalogHandlerUtils, + polarisEventListener); IcebergRestCatalogApi restApi = new IcebergRestCatalogApi(catalogService); IcebergRestConfigurationApi restConfigurationApi = @@ -253,7 +255,8 @@ public String getAuthenticationScheme() { userSecretsManagerFactory, authorizer, callContext, - reservedProperties)); + reservedProperties, + polarisEventListener)); return new TestServices( catalogsApi, From abf8f15af755a95a66708d9a2be5418feed7e6b5 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Mon, 14 Jul 2025 17:39:23 -0700 Subject: [PATCH 02/17] spotlessapply --- ...olarisEclipseLinkMetaStoreManagerTest.java | 2 +- .../jdbc/JdbcBasePersistenceImpl.java | 27 ++-- .../relational/jdbc/models/ModelEvent.java | 1 - .../jdbc/DatasourceOperationsTest.java | 12 +- .../jdbc/models/ModelEventTest.java | 118 +++++++++--------- .../polaris/core/PolarisCallContext.java | 5 +- .../polaris/core/entity/PolarisEvent.java | 2 - .../core/entity/PolarisEventManager.java | 14 +-- .../AtomicOperationMetaStoreManager.java | 3 +- .../TransactionWorkspaceMetaStoreManager.java | 7 +- ...apAtomicOperationMetaStoreManagerTest.java | 2 +- .../PolarisTreeMapMetaStoreManagerTest.java | 2 +- .../config/ProductionReadinessChecks.java | 4 +- .../quarkus/config/QuarkusProducers.java | 7 +- ...rkusPolarisEventListenerConfiguration.java | 1 - .../quarkus/admin/ManagementServiceTest.java | 2 +- .../quarkus/admin/PolarisAuthzTestBase.java | 3 +- .../auth/JWTSymmetricKeyGeneratorTest.java | 3 +- .../quarkus/catalog/IcebergCatalogTest.java | 2 +- .../catalog/IcebergCatalogViewTest.java | 2 +- .../PolarisGenericTableCatalogTest.java | 2 +- .../quarkus/catalog/PolicyCatalogTest.java | 2 +- .../task/TableCleanupTaskHandlerTest.java | 2 +- .../test/PolarisIntegrationTestFixture.java | 2 +- .../test/PolarisIntegrationTestHelper.java | 2 - .../events/AfterTableRefreshedEvent.java | 3 +- .../events/AfterTaskAttemptedEvent.java | 5 +- .../events/AfterViewRefreshedEvent.java | 3 +- .../events/BeforeTableRefreshedEvent.java | 3 +- .../events/BeforeTaskAttemptedEvent.java | 2 - .../events/BeforeViewCommitedEvent.java | 3 +- .../events/BeforeViewRefreshedEvent.java | 3 +- .../events/EventListenerConfiguration.java | 1 + ...BufferPolarisPersistenceEventListener.java | 21 +++- .../service/task/TaskExecutorImpl.java | 6 +- ...erPolarisPersistenceEventListenerTest.java | 59 +++++---- 36 files changed, 185 insertions(+), 153 deletions(-) diff --git a/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java b/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java index acfd6cf89c..2e7328884e 100644 --- a/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java +++ b/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java @@ -97,7 +97,7 @@ protected PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { diagServices, new PolarisConfigurationStore() {}, timeSource.withZone(ZoneId.systemDefault()), - null)); + null)); } @ParameterizedTest diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java index 9ac1c38851..a2d5e62a05 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java @@ -245,14 +245,15 @@ public void writeEvents(@Nonnull List events) { try { // Generate the SQL using the first event as the reference - PreparedQuery firstPreparedQuery = QueryGenerator.generateInsertQuery( + PreparedQuery firstPreparedQuery = + QueryGenerator.generateInsertQuery( ModelEvent.ALL_COLUMNS, ModelEvent.TABLE_NAME, ModelEvent.fromEvent(events.get(0)) - .toMap(datasourceOperations.getDatabaseType()) - .values() - .stream() - .toList(), + .toMap(datasourceOperations.getDatabaseType()) + .values() + .stream() + .toList(), realmId); String expectedSql = firstPreparedQuery.sql(); @@ -262,14 +263,15 @@ public void writeEvents(@Nonnull List events) { // Process remaining events and verify SQL consistency for (int i = 1; i < events.size(); i++) { PolarisEvent event = events.get(i); - PreparedQuery pq = QueryGenerator.generateInsertQuery( + PreparedQuery pq = + QueryGenerator.generateInsertQuery( ModelEvent.ALL_COLUMNS, ModelEvent.TABLE_NAME, ModelEvent.fromEvent(event) - .toMap(datasourceOperations.getDatabaseType()) - .values() - .stream() - .toList(), + .toMap(datasourceOperations.getDatabaseType()) + .values() + .stream() + .toList(), realmId); if (!expectedSql.equals(pq.sql())) { @@ -279,7 +281,8 @@ public void writeEvents(@Nonnull List events) { parametersList.add(pq.parameters()); } - int totalUpdated = datasourceOperations.executeBatchUpdate( + int totalUpdated = + datasourceOperations.executeBatchUpdate( new QueryGenerator.PreparedBatchQuery(expectedSql, parametersList)); if (totalUpdated == 0) { @@ -287,7 +290,7 @@ public void writeEvents(@Nonnull List events) { } } catch (SQLException e) { throw new RuntimeException( - String.format("Failed to write events due to %s", e.getMessage()), e); + String.format("Failed to write events due to %s", e.getMessage()), e); } } diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java index a637c58ea6..ce28cb5fcf 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java @@ -24,7 +24,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; - import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.immutables.PolarisImmutable; import org.apache.polaris.persistence.relational.jdbc.DatabaseType; diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java index b4416e2109..861ca89445 100644 --- a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java @@ -102,7 +102,8 @@ void executeBatchUpdate_success() throws Exception { // There is no way to track how many statements are in a batch, so we are testing how many times // `executeBatch` is being called when(mockDataSource.getConnection()).thenReturn(mockConnection); - String sql = "INSERT INTO POLARIS_SCHEMA.EVENTS (catalog_id, event_id, request_id, event_type, timestamp_ms, principal_name, resource_type, resource_identifier, additional_parameters, realm_id) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ?, ? )"; + String sql = + "INSERT INTO POLARIS_SCHEMA.EVENTS (catalog_id, event_id, request_id, event_type, timestamp_ms, principal_name, resource_type, resource_identifier, additional_parameters, realm_id) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ?, ? )"; List> queryParams = new ArrayList<>(); for (int i = 0; i < 1000; i++) { ModelEvent modelEvent = @@ -118,14 +119,17 @@ void executeBatchUpdate_success() throws Exception { .additionalParameters("") .build(); queryParams.add( - modelEvent.toMap(datasourceOperations.getDatabaseType()).values().stream().toList()); + modelEvent.toMap(datasourceOperations.getDatabaseType()).values().stream().toList()); } when(mockConnection.prepareStatement(any())).thenReturn(mockPreparedStatement); when(mockPreparedStatement.executeBatch()).thenReturn(new int[] {100}); - int result = datasourceOperations.executeBatchUpdate(new QueryGenerator.PreparedBatchQuery(sql, queryParams)); + int result = + datasourceOperations.executeBatchUpdate( + new QueryGenerator.PreparedBatchQuery(sql, queryParams)); assertEquals( - queryParams.size() + 100, result); // ExecuteBatch will be called once more than actual batches + queryParams.size() + 100, + result); // ExecuteBatch will be called once more than actual batches } @Test diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java index d0b5a21112..a1a2117c03 100644 --- a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java @@ -27,7 +27,6 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.Map; - import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.persistence.relational.jdbc.DatabaseType; import org.junit.jupiter.api.Test; @@ -53,7 +52,8 @@ public class ModelEventTest { private static final String TEST_EVENT_TYPE = "CREATE"; private static final long TEST_TIMESTAMP_MS = 1234567890L; private static final String TEST_USER = "test-user"; - private static final PolarisEvent.ResourceType TEST_RESOURCE_TYPE = PolarisEvent.ResourceType.TABLE; + private static final PolarisEvent.ResourceType TEST_RESOURCE_TYPE = + PolarisEvent.ResourceType.TABLE; private static final String TEST_RESOURCE_TYPE_STRING = "TABLE"; private static final String TEST_RESOURCE_IDENTIFIER = "test-table"; private static final String EMPTY_JSON = "{}"; @@ -62,7 +62,8 @@ public class ModelEventTest { // Dummy values for test initialization private static final String DUMMY = "dummy"; private static final long DUMMY_TIMESTAMP = 0L; - private static final PolarisEvent.ResourceType DUMMY_RESOURCE_TYPE = PolarisEvent.ResourceType.CATALOG; + private static final PolarisEvent.ResourceType DUMMY_RESOURCE_TYPE = + PolarisEvent.ResourceType.CATALOG; @Test public void testFromResultSet() throws SQLException { @@ -79,17 +80,18 @@ public void testFromResultSet() throws SQLException { when(mockResultSet.getString(ADDITIONAL_PARAMETERS)).thenReturn(EMPTY_JSON); // Create a concrete implementation of ModelEvent for testing - ModelEvent modelEvent = ImmutableModelEvent.builder() - .catalogId(DUMMY) - .eventId(DUMMY) - .requestId(DUMMY) - .eventType(DUMMY) - .timestampMs(DUMMY_TIMESTAMP) - .principalName(DUMMY) - .resourceType(DUMMY_RESOURCE_TYPE) - .resourceIdentifier(DUMMY) - .additionalParameters(EMPTY_JSON) - .build(); + ModelEvent modelEvent = + ImmutableModelEvent.builder() + .catalogId(DUMMY) + .eventId(DUMMY) + .requestId(DUMMY) + .eventType(DUMMY) + .timestampMs(DUMMY_TIMESTAMP) + .principalName(DUMMY) + .resourceType(DUMMY_RESOURCE_TYPE) + .resourceIdentifier(DUMMY) + .additionalParameters(EMPTY_JSON) + .build(); // Act PolarisEvent result = modelEvent.fromResultSet(mockResultSet); @@ -109,17 +111,18 @@ public void testFromResultSet() throws SQLException { @Test public void testToMapWithH2DatabaseType() { // Arrange - ModelEvent modelEvent = ImmutableModelEvent.builder() - .catalogId(TEST_CATALOG_ID) - .eventId(TEST_EVENT_ID) - .requestId(TEST_REQUEST_ID) - .eventType(TEST_EVENT_TYPE) - .timestampMs(TEST_TIMESTAMP_MS) - .principalName(TEST_USER) - .resourceType(TEST_RESOURCE_TYPE) - .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) - .additionalParameters(TEST_JSON) - .build(); + ModelEvent modelEvent = + ImmutableModelEvent.builder() + .catalogId(TEST_CATALOG_ID) + .eventId(TEST_EVENT_ID) + .requestId(TEST_REQUEST_ID) + .eventType(TEST_EVENT_TYPE) + .timestampMs(TEST_TIMESTAMP_MS) + .principalName(TEST_USER) + .resourceType(TEST_RESOURCE_TYPE) + .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) + .additionalParameters(TEST_JSON) + .build(); // Act Map resultMap = modelEvent.toMap(DatabaseType.H2); @@ -139,17 +142,18 @@ public void testToMapWithH2DatabaseType() { @Test public void testToMapWithPostgresType() { // Arrange - ModelEvent modelEvent = ImmutableModelEvent.builder() - .catalogId(TEST_CATALOG_ID) - .eventId(TEST_EVENT_ID) - .requestId(TEST_REQUEST_ID) - .eventType(TEST_EVENT_TYPE) - .timestampMs(TEST_TIMESTAMP_MS) - .principalName(TEST_USER) - .resourceType(TEST_RESOURCE_TYPE) - .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) - .additionalParameters(TEST_JSON) - .build(); + ModelEvent modelEvent = + ImmutableModelEvent.builder() + .catalogId(TEST_CATALOG_ID) + .eventId(TEST_EVENT_ID) + .requestId(TEST_REQUEST_ID) + .eventType(TEST_EVENT_TYPE) + .timestampMs(TEST_TIMESTAMP_MS) + .principalName(TEST_USER) + .resourceType(TEST_RESOURCE_TYPE) + .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) + .additionalParameters(TEST_JSON) + .build(); // Act Map resultMap = modelEvent.toMap(DatabaseType.POSTGRES); @@ -182,15 +186,16 @@ public void testFromEventWithNullInput() { @Test public void testFromEvent() { // Arrange - PolarisEvent polarisEvent = new PolarisEvent( - TEST_CATALOG_ID, - TEST_EVENT_ID, - TEST_REQUEST_ID, - TEST_EVENT_TYPE, - TEST_TIMESTAMP_MS, - TEST_USER, - TEST_RESOURCE_TYPE, - TEST_RESOURCE_IDENTIFIER); + PolarisEvent polarisEvent = + new PolarisEvent( + TEST_CATALOG_ID, + TEST_EVENT_ID, + TEST_REQUEST_ID, + TEST_EVENT_TYPE, + TEST_TIMESTAMP_MS, + TEST_USER, + TEST_RESOURCE_TYPE, + TEST_RESOURCE_IDENTIFIER); polarisEvent.setAdditionalParameters(TEST_JSON); // Act @@ -220,17 +225,18 @@ public void testToEventWithNullInput() { @Test public void testToEvent() { // Arrange - ModelEvent modelEvent = ImmutableModelEvent.builder() - .catalogId(TEST_CATALOG_ID) - .eventId(TEST_EVENT_ID) - .requestId(TEST_REQUEST_ID) - .eventType(TEST_EVENT_TYPE) - .timestampMs(TEST_TIMESTAMP_MS) - .principalName(TEST_USER) - .resourceType(TEST_RESOURCE_TYPE) - .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) - .additionalParameters(TEST_JSON) - .build(); + ModelEvent modelEvent = + ImmutableModelEvent.builder() + .catalogId(TEST_CATALOG_ID) + .eventId(TEST_EVENT_ID) + .requestId(TEST_REQUEST_ID) + .eventType(TEST_EVENT_TYPE) + .timestampMs(TEST_TIMESTAMP_MS) + .principalName(TEST_USER) + .resourceType(TEST_RESOURCE_TYPE) + .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) + .additionalParameters(TEST_JSON) + .build(); // Act PolarisEvent result = ModelEvent.toEvent(modelEvent); diff --git a/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java b/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java index 90f930e176..fedb03a979 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java @@ -19,11 +19,10 @@ package org.apache.polaris.core; import jakarta.annotation.Nonnull; +import jakarta.ws.rs.container.ContainerRequestContext; import java.time.Clock; import java.time.ZoneId; import java.util.UUID; - -import jakarta.ws.rs.container.ContainerRequestContext; import org.apache.polaris.core.config.PolarisConfigurationStore; import org.apache.polaris.core.config.RealmConfig; import org.apache.polaris.core.config.RealmConfigImpl; @@ -87,7 +86,7 @@ public PolarisCallContext( diagServices, new PolarisConfigurationStore() {}, Clock.system(ZoneId.systemDefault()), - null); + null); } public BasePersistence getMetaStore() { diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java index 63779a26f1..7e616de139 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java @@ -19,9 +19,7 @@ package org.apache.polaris.core.entity; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import java.util.Map; diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEventManager.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEventManager.java index 2acae49cf3..b4b7891338 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEventManager.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEventManager.java @@ -20,15 +20,15 @@ package org.apache.polaris.core.entity; import jakarta.annotation.Nonnull; +import java.util.List; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.persistence.BasePersistence; -import java.util.List; - public interface PolarisEventManager { - @Nonnull - default void writeEvents(@Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { - BasePersistence ms = callCtx.getMetaStore(); - ms.writeEvents(polarisEvents); - } + @Nonnull + default void writeEvents( + @Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { + BasePersistence ms = callCtx.getMetaStore(); + ms.writeEvents(polarisEvents); + } } diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java index dcb7cb3e74..5a2d50f749 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java @@ -1974,7 +1974,8 @@ private List loadPoliciesFromMappingRecords( @Nonnull @Override - public void writeEvents(@Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { + public void writeEvents( + @Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { // get metastore we should be using BasePersistence ms = callCtx.getMetaStore(); diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/TransactionWorkspaceMetaStoreManager.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/TransactionWorkspaceMetaStoreManager.java index 1218e518e8..beae726310 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/TransactionWorkspaceMetaStoreManager.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/TransactionWorkspaceMetaStoreManager.java @@ -448,9 +448,8 @@ Optional> hasOverlappingSiblings( @Nonnull @Override - public void writeEvents(@Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { - callCtx - .getDiagServices() - .fail("illegal_method_in_transaction_workspace", "writeEvents"); + public void writeEvents( + @Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { + callCtx.getDiagServices().fail("illegal_method_in_transaction_workspace", "writeEvents"); } } diff --git a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java index 7d2b2dd0ba..6c65a2ef48 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java @@ -42,7 +42,7 @@ public PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { diagServices, new PolarisConfigurationStore() {}, timeSource.withZone(ZoneId.systemDefault()), - null); + null); return new PolarisTestMetaStoreManager(new AtomicOperationMetaStoreManager(), callCtx); } diff --git a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java index 8e390748b0..462c4b58a3 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java @@ -42,7 +42,7 @@ public PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { diagServices, new PolarisConfigurationStore() {}, timeSource.withZone(ZoneId.systemDefault()), - null); + null); return new PolarisTestMetaStoreManager(new TransactionalMetaStoreManagerImpl(), callCtx); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/ProductionReadinessChecks.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/ProductionReadinessChecks.java index 78ffaf1e98..bb6095a3ee 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/ProductionReadinessChecks.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/ProductionReadinessChecks.java @@ -202,7 +202,9 @@ public ProductionReadinessCheck checkPolarisEventListener( PolarisEventListener polarisEventListener) { if (polarisEventListener instanceof TestPolarisEventListener) { return ProductionReadinessCheck.of( - Error.of("TestPolarisEventListener is intended for tests only.", "polaris.event-listener.type")); + Error.of( + "TestPolarisEventListener is intended for tests only.", + "polaris.event-listener.type")); } return ProductionReadinessCheck.OK; } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java index 2a025e0b25..5f35a5efeb 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java @@ -136,7 +136,12 @@ public CallContext polarisCallContext( BasePersistence metaStoreSession = metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(); return new PolarisCallContext( - realmContext, metaStoreSession, diagServices, configurationStore, clock, containerRequestContext); + realmContext, + metaStoreSession, + diagServices, + configurationStore, + clock, + containerRequestContext); } @Produces diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java index 9a6adc2f83..2a5452ceb7 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java @@ -20,7 +20,6 @@ import io.quarkus.runtime.annotations.StaticInitSafe; import io.smallrye.config.ConfigMapping; - import java.time.Duration; import java.util.Optional; import org.apache.polaris.service.events.EventListenerConfiguration; diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java index 1c6d4fa06b..b9aa7cf758 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java @@ -86,7 +86,7 @@ public void setup() { fakeServices.polarisDiagnostics(), fakeServices.configurationStore(), Mockito.mock(Clock.class), - null); + null); CallContext.setCurrentContext(polarisCallContext); services = TestServices.builder() diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/PolarisAuthzTestBase.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/PolarisAuthzTestBase.java index 6b2f27b527..c342708f26 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/PolarisAuthzTestBase.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/PolarisAuthzTestBase.java @@ -234,7 +234,8 @@ public void before(TestInfo testInfo) { RealmContext realmContext = testInfo::getDisplayName; QuarkusMock.installMockForType(realmContext, RealmContext.class); ContainerRequestContext containerRequestContext = Mockito.mock(ContainerRequestContext.class); - Mockito.when(containerRequestContext.getProperty(Mockito.anyString())).thenReturn("request-id-1"); + Mockito.when(containerRequestContext.getProperty(Mockito.anyString())) + .thenReturn("request-id-1"); QuarkusMock.installMockForType(containerRequestContext, ContainerRequestContext.class); metaStoreManager = managerFactory.getOrCreateMetaStoreManager(realmContext); userSecretsManager = userSecretsManagerFactory.getOrCreateUserSecretsManager(realmContext); diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java index ff8a734584..9a6c40c8f5 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java @@ -45,7 +45,8 @@ public class JWTSymmetricKeyGeneratorTest { /** Sanity test to verify that we can generate a token */ @Test public void testJWTSymmetricKeyGenerator() { - PolarisCallContext polarisCallContext = new PolarisCallContext(null, null, null, null, null, null); + PolarisCallContext polarisCallContext = + new PolarisCallContext(null, null, null, null, null, null); PolarisMetaStoreManager metastoreManager = Mockito.mock(PolarisMetaStoreManager.class); String mainSecret = "test_secret"; String clientId = "test_client_id"; diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java index 6bcd23bd84..2fa410c4ba 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java @@ -285,7 +285,7 @@ public void before(TestInfo testInfo) { diagServices, configurationStore, Clock.systemDefaultZone(), - null); + null); entityManager = new PolarisEntityManager( diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogViewTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogViewTest.java index 014b155e1e..e389979bfd 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogViewTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogViewTest.java @@ -175,7 +175,7 @@ public void before(TestInfo testInfo) { diagServices, configurationStore, Clock.systemDefaultZone(), - null); + null); PolarisEntityManager entityManager = new PolarisEntityManager( diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolarisGenericTableCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolarisGenericTableCatalogTest.java index 6eac9f213e..feb6c9aef0 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolarisGenericTableCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolarisGenericTableCatalogTest.java @@ -168,7 +168,7 @@ public void before(TestInfo testInfo) { diagServices, configurationStore, Clock.systemDefaultZone(), - null); + null); entityManager = new PolarisEntityManager( metaStoreManager, diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolicyCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolicyCatalogTest.java index 562ac3a3ff..08ef3a2e9b 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolicyCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/PolicyCatalogTest.java @@ -194,7 +194,7 @@ public void before(TestInfo testInfo) { diagServices, configurationStore, Clock.systemDefaultZone(), - null); + null); entityManager = new PolarisEntityManager( metaStoreManager, diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java index 4a683bb19e..31817aae32 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java @@ -107,7 +107,7 @@ void setup() { diagServices, configurationStore, Clock.systemDefaultZone(), - null); + null); } @Test diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java index e3999b1578..a33e43fdbb 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java @@ -119,7 +119,7 @@ private PolarisPrincipalSecrets fetchAdminSecrets() { helper.diagServices, helper.configurationStore, helper.clock, - null); + null); try { PolarisMetaStoreManager metaStoreManager = helper.metaStoreManagerFactory.getOrCreateMetaStoreManager(realmContext); diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestHelper.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestHelper.java index e93c492f50..7149291543 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestHelper.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestHelper.java @@ -22,8 +22,6 @@ import jakarta.inject.Inject; import jakarta.inject.Singleton; import java.time.Clock; - -import jakarta.ws.rs.container.ContainerRequestContext; import org.apache.polaris.core.PolarisDiagnostics; import org.apache.polaris.core.config.PolarisConfigurationStore; import org.apache.polaris.core.persistence.MetaStoreManagerFactory; diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java index d243789596..00567b4a86 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java @@ -26,4 +26,5 @@ * @param eventId The event unique identifier * @param tableIdentifier The identifier of the table that was refreshed. */ -public record AfterTableRefreshedEvent(String eventId, TableIdentifier tableIdentifier) implements PolarisEvent {} +public record AfterTableRefreshedEvent(String eventId, TableIdentifier tableIdentifier) + implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java index 9ae922d983..a3e132e0cd 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java @@ -18,8 +18,6 @@ */ package org.apache.polaris.service.events; -import org.apache.polaris.core.context.CallContext; - /** * Emitted after an attempt of an async task, such as manifest file cleanup, finishes. * @@ -29,5 +27,4 @@ * @param success Whether the attempt succeeded. */ public record AfterTaskAttemptedEvent( - String eventId, long taskEntityId, int attempt, boolean success) - implements PolarisEvent {} + String eventId, long taskEntityId, int attempt, boolean success) implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java index 4a11272e3f..edb13d2b20 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java @@ -25,4 +25,5 @@ * * @param viewIdentifier The identifier of the view that was refreshed. */ -public record AfterViewRefreshedEvent(String eventId, TableIdentifier viewIdentifier) implements PolarisEvent {} +public record AfterViewRefreshedEvent(String eventId, TableIdentifier viewIdentifier) + implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java index bf350ca87e..b50d1abb6b 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java @@ -26,4 +26,5 @@ * * @param tableIdentifier The identifier of the table being refreshed. */ -public record BeforeTableRefreshedEvent(String eventId, TableIdentifier tableIdentifier) implements PolarisEvent {} +public record BeforeTableRefreshedEvent(String eventId, TableIdentifier tableIdentifier) + implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java index 03a807ad4b..13ecb0ce2e 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java @@ -18,8 +18,6 @@ */ package org.apache.polaris.service.events; -import org.apache.polaris.core.context.CallContext; - /** * Emitted before an attempt of an async task, such as manifest file cleanup, begins. * diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java index 2fa8296c98..c163cc5716 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java @@ -31,4 +31,5 @@ * @param metadata The new metadata. */ public record BeforeViewCommitedEvent( - String eventId, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) implements PolarisEvent {} + String eventId, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) + implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java b/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java index 3ff290a629..1b652690a9 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java @@ -26,4 +26,5 @@ * * @param viewIdentifier The identifier of the view being refreshed. */ -public record BeforeViewRefreshedEvent(String eventId, TableIdentifier viewIdentifier) implements PolarisEvent {} +public record BeforeViewRefreshedEvent(String eventId, TableIdentifier viewIdentifier) + implements PolarisEvent {} diff --git a/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java b/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java index 6e91f79e36..c3ca6cb0b5 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java @@ -26,6 +26,7 @@ * Configuration for event listeners. * *

bufferTime() specifies the buffer time in milliseconds. + * *

maxBufferSize() specifies the maximum number of cached entries. */ public interface EventListenerConfiguration { diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java index e028f86fd9..5deac1c5c8 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java @@ -34,7 +34,6 @@ import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.entity.PolarisEvent; @@ -52,7 +51,8 @@ public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersis private final MetaStoreManagerFactory metaStoreManagerFactory; private final Clock clock; - private final ConcurrentHashMap> buffer = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> buffer = + new ConcurrentHashMap<>(); private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); private final ConcurrentHashMap, Integer> futures = new ConcurrentHashMap<>(); private final Duration timeToFlush; @@ -75,7 +75,9 @@ public InMemoryBufferPolarisPersistenceEventListener( @PostConstruct void start() { futures.put( - executor.scheduleAtFixedRate(this::runCleanup, 0, timeToFlush.toMillis(), TimeUnit.MILLISECONDS), 1); + executor.scheduleAtFixedRate( + this::runCleanup, 0, timeToFlush.toMillis(), TimeUnit.MILLISECONDS), + 1); } void runCleanup() { @@ -104,7 +106,9 @@ void shutdown() { void addToBuffer(PolarisEvent polarisEvent, CallContext callCtx) { String realmId = callCtx.getRealmContext().getRealmIdentifier(); - buffer.computeIfAbsent(realmId, k -> new ConcurrentLinkedQueue<>()).add(new EventAndContext(polarisEvent, callCtx.getPolarisCallContext().copy())); + buffer + .computeIfAbsent(realmId, k -> new ConcurrentLinkedQueue<>()) + .add(new EventAndContext(polarisEvent, callCtx.getPolarisCallContext().copy())); futures.put(executor.submit(() -> checkAndFlushBufferIfNecessary(realmId)), 1); } @@ -115,7 +119,8 @@ public void checkAndFlushBufferIfNecessary(String realmId) { return; } - // Given that we are using a ConcurrentLinkedQueue, this should not lock any calls to `add` on the queue. + // Given that we are using a ConcurrentLinkedQueue, this should not lock any calls to `add` on + // the queue. synchronized (queue) { // Double-check inside synchronized block if (queue.isEmpty()) { @@ -137,7 +142,11 @@ public void checkAndFlushBufferIfNecessary(String realmId) { return; } - metaStoreManagerFactory.getOrCreateMetaStoreManager(() -> realmId).writeEvents(head.callContext(), new ArrayList<>(queue.stream().map(EventAndContext::polarisEvent).toList())); + metaStoreManagerFactory + .getOrCreateMetaStoreManager(() -> realmId) + .writeEvents( + head.callContext(), + new ArrayList<>(queue.stream().map(EventAndContext::polarisEvent).toList())); } } } diff --git a/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java b/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java index 09b2fd487e..0f7b255f23 100644 --- a/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java +++ b/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java @@ -119,8 +119,7 @@ public void addTaskHandlerContext(long taskEntityId, CallContext callContext) { protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { polarisEventListener.onBeforeTaskAttempted( - new BeforeTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt), - ctx); + new BeforeTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt), ctx); boolean success = false; try { @@ -166,8 +165,7 @@ protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { } } finally { polarisEventListener.onAfterTaskAttempted( - new AfterTaskAttemptedEvent( - PolarisEvent.createEventId(), taskEntityId, attempt, success), + new AfterTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt, success), ctx); } } diff --git a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index 21c4d34f48..83de739dc0 100644 --- a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -34,7 +34,6 @@ import java.util.Optional; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; - import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.context.RealmContext; @@ -92,7 +91,8 @@ public void testAddToBufferFlushesAfterConfiguredTime() { // Push clock forwards to flush the buffer clock.add(CONFIG_TIME_TO_FLUSH_IN_MS.multipliedBy(2)); eventListener.checkAndFlushBufferIfNecessary(realmId); - verify(polarisMetaStoreManager, times(1)).writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); + verify(polarisMetaStoreManager, times(1)) + .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); } @Test @@ -108,12 +108,16 @@ public void testAddToBufferFlushesAfterMaxEvents() { eventListener.addToBuffer(triggeringEvent, callContext); eventsAddedToBuffer.add(triggeringEvent); - // Given the call to checkAndFlushBufferIfNecessary is async, the calling thread should not have blocked and nothing would've been done immediately - verify(polarisMetaStoreManager, times(0)).writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); + // Given the call to checkAndFlushBufferIfNecessary is async, the calling thread should not have + // blocked and nothing would've been done immediately + verify(polarisMetaStoreManager, times(0)) + .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); - // Calling checkAndFlushBufferIfNecessary manually to replicate the behavior of the executor service + // Calling checkAndFlushBufferIfNecessary manually to replicate the behavior of the executor + // service eventListener.checkAndFlushBufferIfNecessary(realm1); - verify(polarisMetaStoreManager, times(0)).writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); + verify(polarisMetaStoreManager, times(0)) + .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); } @Test @@ -131,13 +135,15 @@ public void testCheckAndFlushBufferIfNecessaryIsThreadSafe() throws Exception { // Each thread will call checkAndFlushBufferIfNecessary concurrently for (int i = 0; i < threadCount; i++) { - Thread t = new Thread(() -> { - try { - eventListener.checkAndFlushBufferIfNecessary(realmId); - } catch (Exception e) { - exceptions.add(e); - } - }); + Thread t = + new Thread( + () -> { + try { + eventListener.checkAndFlushBufferIfNecessary(realmId); + } catch (Exception e) { + exceptions.add(e); + } + }); threads.add(t); } // Start all threads @@ -148,10 +154,12 @@ public void testCheckAndFlushBufferIfNecessaryIsThreadSafe() throws Exception { } // There should be no exceptions if (!exceptions.isEmpty()) { - throw new AssertionError("Exceptions occurred in concurrent checkAndFlushBufferIfNecessary: ", exceptions.peek()); + throw new AssertionError( + "Exceptions occurred in concurrent checkAndFlushBufferIfNecessary: ", exceptions.peek()); } // Only one flush should occur - verify(polarisMetaStoreManager, times(1)).writeEvents(eq(callContext.getPolarisCallContext()), eq(events)); + verify(polarisMetaStoreManager, times(1)) + .writeEvents(eq(callContext.getPolarisCallContext()), eq(events)); } private List addEventsWithoutTriggeringFlush(String realmId) { @@ -162,7 +170,7 @@ private List addEventsWithoutTriggeringFlush(String realmId) { RealmContext realmContext = () -> realmId; when(callContext.getRealmContext()).thenReturn(realmContext); for (PolarisEvent realmEvent : realmEvents) { - eventListener.addToBuffer(realmEvent, callContext); + eventListener.addToBuffer(realmEvent, callContext); } verify(polarisMetaStoreManager, times(0)).writeEvents(Mockito.any(), Mockito.any()); return realmEvents; @@ -178,15 +186,16 @@ private PolarisEvent createSampleEvent() { PolarisEvent.ResourceType resourceType = PolarisEvent.ResourceType.TABLE; String resourceIdentifier = "test-table"; - PolarisEvent event = new PolarisEvent( - catalogId, - id, - requestId, - eventType, - timestampMs, - principalName, - resourceType, - resourceIdentifier); + PolarisEvent event = + new PolarisEvent( + catalogId, + id, + requestId, + eventType, + timestampMs, + principalName, + resourceType, + resourceIdentifier); Map additionalParams = new HashMap<>(); additionalParams.put("testKey", "testValue"); From 2ff9560886a2b0496ba027a6d35d994ea3bc15ba Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Mon, 14 Jul 2025 18:45:08 -0700 Subject: [PATCH 03/17] modify test case --- ...moryBufferPolarisPersistenceEventListenerTest.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index 83de739dc0..5fc2bae2f4 100644 --- a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -99,7 +99,7 @@ public void testAddToBufferFlushesAfterConfiguredTime() { public void testAddToBufferFlushesAfterMaxEvents() { String realm1 = "realm1"; List eventsAddedToBuffer = addEventsWithoutTriggeringFlush(realm1); - addEventsWithoutTriggeringFlush("realm2"); + List eventsAddedToBufferRealm2 = addEventsWithoutTriggeringFlush("realm2"); // Add the last event for realm1 and verify that it did trigger the flush PolarisEvent triggeringEvent = createSampleEvent(); @@ -108,16 +108,13 @@ public void testAddToBufferFlushesAfterMaxEvents() { eventListener.addToBuffer(triggeringEvent, callContext); eventsAddedToBuffer.add(triggeringEvent); - // Given the call to checkAndFlushBufferIfNecessary is async, the calling thread should not have - // blocked and nothing would've been done immediately - verify(polarisMetaStoreManager, times(0)) - .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); - // Calling checkAndFlushBufferIfNecessary manually to replicate the behavior of the executor // service eventListener.checkAndFlushBufferIfNecessary(realm1); - verify(polarisMetaStoreManager, times(0)) + verify(polarisMetaStoreManager, times(1)) .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); + verify(polarisMetaStoreManager, times(0)) + .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBufferRealm2)); } @Test From c9430b3c0bb2effaddab00dc346cffd08ba9d10d Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Mon, 14 Jul 2025 19:06:29 -0700 Subject: [PATCH 04/17] spotlessapply --- .../InMemoryBufferPolarisPersistenceEventListenerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index 5fc2bae2f4..ec570deeea 100644 --- a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -114,7 +114,7 @@ public void testAddToBufferFlushesAfterMaxEvents() { verify(polarisMetaStoreManager, times(1)) .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); verify(polarisMetaStoreManager, times(0)) - .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBufferRealm2)); + .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBufferRealm2)); } @Test From 51a943491164c9ee45d10cbacf97c560b2907b85 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Wed, 23 Jul 2025 03:36:14 -0700 Subject: [PATCH 05/17] revision based on comments from @eric-maynard --- .../relational/jdbc/models/ModelEvent.java | 14 +-- .../src/main/resources/h2/schema-v1.sql | 14 --- .../src/main/resources/h2/schema-v3.sql | 2 +- .../src/main/resources/postgres/schema-v3.sql | 2 +- .../jdbc/DatasourceOperationsTest.java | 4 +- .../jdbc/models/ModelEventTest.java | 26 ++--- .../polaris/core/PolarisCallContext.java | 23 +---- .../polaris/core/entity/PolarisEvent.java | 14 +-- .../src/main/resources/application.properties | 9 +- .../quarkus/config/QuarkusProducers.java | 6 +- ...rkusPolarisEventListenerConfiguration.java | 11 +-- ...emoryBufferEventListenerConfiguration.java | 51 ++++++++++ .../context/DefaultCallContextResolver.java | 2 +- ...fferPersistenceListenerConfiguration.java} | 4 +- ...BufferPolarisPersistenceEventListener.java | 19 +++- .../PolarisPersistenceEventListener.java | 10 +- .../service/catalog/io/FileIOFactoryTest.java | 3 +- ...erPolarisPersistenceEventListenerTest.java | 98 ++++++++++++++++++- .../apache/polaris/service/TestServices.java | 3 +- 19 files changed, 213 insertions(+), 102 deletions(-) create mode 100644 runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java rename service/common/src/main/java/org/apache/polaris/service/events/{EventListenerConfiguration.java => listeners/InMemoryBufferPersistenceListenerConfiguration.java} (91%) diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java index ce28cb5fcf..8700cff8bf 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEvent.java @@ -42,7 +42,7 @@ public interface ModelEvent extends Converter { "principal_name", "resource_type", "resource_identifier", - "additional_parameters"); + "additional_properties"); // catalog id String getCatalogId(); @@ -69,7 +69,7 @@ public interface ModelEvent extends Converter { String getResourceIdentifier(); // Additional parameters that were not earlier recorded - String getAdditionalParameters(); + String getAdditionalProperties(); @Override default PolarisEvent fromResultSet(ResultSet rs) throws SQLException { @@ -83,7 +83,7 @@ default PolarisEvent fromResultSet(ResultSet rs) throws SQLException { .principalName(rs.getString("actor")) .resourceType(PolarisEvent.ResourceType.valueOf(rs.getString("resource_type"))) .resourceIdentifier(rs.getString("resource_identifier")) - .additionalParameters(rs.getString("additional_parameters")) + .additionalProperties(rs.getString("additional_properties")) .build(); return toEvent(modelEvent); } @@ -100,9 +100,9 @@ default Map toMap(DatabaseType databaseType) { map.put("resource_type", getResourceType().toString()); map.put("resource_identifier", getResourceIdentifier()); if (databaseType.equals(DatabaseType.POSTGRES)) { - map.put("additional_parameters", toJsonbPGobject(getAdditionalParameters())); + map.put("additional_properties", toJsonbPGobject(getAdditionalProperties())); } else { - map.put("additional_parameters", getAdditionalParameters()); + map.put("additional_properties", getAdditionalProperties()); } return map; } @@ -119,7 +119,7 @@ static ModelEvent fromEvent(PolarisEvent event) { .principalName(event.getPrincipalName()) .resourceType(event.getResourceType()) .resourceIdentifier(event.getResourceIdentifier()) - .additionalParameters(event.getAdditionalParameters()) + .additionalProperties(event.getAdditionalProperties()) .build(); } @@ -136,7 +136,7 @@ static PolarisEvent toEvent(ModelEvent model) { model.getPrincipalName(), model.getResourceType(), model.getResourceIdentifier()); - polarisEvent.setAdditionalParameters(model.getAdditionalParameters()); + polarisEvent.setAdditionalProperties(model.getAdditionalProperties()); return polarisEvent; } } diff --git a/persistence/relational-jdbc/src/main/resources/h2/schema-v1.sql b/persistence/relational-jdbc/src/main/resources/h2/schema-v1.sql index 734462be91..10b4e3774b 100644 --- a/persistence/relational-jdbc/src/main/resources/h2/schema-v1.sql +++ b/persistence/relational-jdbc/src/main/resources/h2/schema-v1.sql @@ -118,17 +118,3 @@ CREATE TABLE IF NOT EXISTS policy_mapping_record ( ); CREATE INDEX IF NOT EXISTS idx_policy_mapping_record ON policy_mapping_record (realm_id, policy_type_code, policy_catalog_id, policy_id, target_catalog_id, target_id); - -CREATE TABLE IF NOT EXISTS events ( - realm_id TEXT NOT NULL, - catalog_id TEXT NOT NULL, - event_id TEXT NOT NULL, - request_id TEXT NOT NULL, - event_type TEXT NOT NULL, - timestamp_ms BIGINT NOT NULL, - principal_name TEXT, - resource_type TEXT NOT NULL, - resource_identifier TEXT NOT NULL, - additional_parameters TEXT NOT NULL DEFAULT '{}', - PRIMARY KEY (event_id) -); diff --git a/persistence/relational-jdbc/src/main/resources/h2/schema-v3.sql b/persistence/relational-jdbc/src/main/resources/h2/schema-v3.sql index 5bdf52a565..947799246a 100644 --- a/persistence/relational-jdbc/src/main/resources/h2/schema-v3.sql +++ b/persistence/relational-jdbc/src/main/resources/h2/schema-v3.sql @@ -135,6 +135,6 @@ CREATE TABLE IF NOT EXISTS events ( principal_name TEXT, resource_type TEXT NOT NULL, resource_identifier TEXT NOT NULL, - additional_parameters JSONB NOT NULL DEFAULT '{}'::JSONB, + additional_properties JSONB NOT NULL DEFAULT '{}'::JSONB, PRIMARY KEY (event_id) ); \ No newline at end of file diff --git a/persistence/relational-jdbc/src/main/resources/postgres/schema-v3.sql b/persistence/relational-jdbc/src/main/resources/postgres/schema-v3.sql index 628eef84b4..38a1ee4927 100644 --- a/persistence/relational-jdbc/src/main/resources/postgres/schema-v3.sql +++ b/persistence/relational-jdbc/src/main/resources/postgres/schema-v3.sql @@ -132,6 +132,6 @@ CREATE TABLE IF NOT EXISTS events ( principal_name TEXT, resource_type TEXT NOT NULL, resource_identifier TEXT NOT NULL, - additional_parameters JSONB NOT NULL DEFAULT '{}'::JSONB, + additional_properties JSONB NOT NULL DEFAULT '{}'::JSONB, PRIMARY KEY (event_id) ); diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java index 861ca89445..179aa6879d 100644 --- a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/DatasourceOperationsTest.java @@ -103,7 +103,7 @@ void executeBatchUpdate_success() throws Exception { // `executeBatch` is being called when(mockDataSource.getConnection()).thenReturn(mockConnection); String sql = - "INSERT INTO POLARIS_SCHEMA.EVENTS (catalog_id, event_id, request_id, event_type, timestamp_ms, principal_name, resource_type, resource_identifier, additional_parameters, realm_id) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ?, ? )"; + "INSERT INTO POLARIS_SCHEMA.EVENTS (catalog_id, event_id, request_id, event_type, timestamp_ms, principal_name, resource_type, resource_identifier, additional_properties, realm_id) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ?, ? )"; List> queryParams = new ArrayList<>(); for (int i = 0; i < 1000; i++) { ModelEvent modelEvent = @@ -116,7 +116,7 @@ void executeBatchUpdate_success() throws Exception { .eventType("event_type1") .timestampMs(1234) .principalName("principal_" + i) - .additionalParameters("") + .additionalProperties("") .build(); queryParams.add( modelEvent.toMap(datasourceOperations.getDatabaseType()).values().stream().toList()); diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java index a1a2117c03..fa78de0885 100644 --- a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelEventTest.java @@ -43,7 +43,7 @@ public class ModelEventTest { private static final String PRINCIPAL_NAME = "principal_name"; private static final String RESOURCE_TYPE = "resource_type"; private static final String RESOURCE_IDENTIFIER = "resource_identifier"; - private static final String ADDITIONAL_PARAMETERS = "additional_parameters"; + private static final String ADDITIONAL_PROPERTIES = "additional_properties"; // Test data values private static final String TEST_CATALOG_ID = "test-catalog"; @@ -77,7 +77,7 @@ public void testFromResultSet() throws SQLException { when(mockResultSet.getString(ACTOR)).thenReturn(TEST_USER); when(mockResultSet.getString(RESOURCE_TYPE)).thenReturn(TEST_RESOURCE_TYPE_STRING); when(mockResultSet.getString(RESOURCE_IDENTIFIER)).thenReturn(TEST_RESOURCE_IDENTIFIER); - when(mockResultSet.getString(ADDITIONAL_PARAMETERS)).thenReturn(EMPTY_JSON); + when(mockResultSet.getString(ADDITIONAL_PROPERTIES)).thenReturn(EMPTY_JSON); // Create a concrete implementation of ModelEvent for testing ModelEvent modelEvent = @@ -90,7 +90,7 @@ public void testFromResultSet() throws SQLException { .principalName(DUMMY) .resourceType(DUMMY_RESOURCE_TYPE) .resourceIdentifier(DUMMY) - .additionalParameters(EMPTY_JSON) + .additionalProperties(EMPTY_JSON) .build(); // Act @@ -105,7 +105,7 @@ public void testFromResultSet() throws SQLException { assertEquals(TEST_USER, result.getPrincipalName()); assertEquals(TEST_RESOURCE_TYPE, result.getResourceType()); assertEquals(TEST_RESOURCE_IDENTIFIER, result.getResourceIdentifier()); - assertEquals(EMPTY_JSON, result.getAdditionalParameters()); + assertEquals(EMPTY_JSON, result.getAdditionalProperties()); } @Test @@ -121,7 +121,7 @@ public void testToMapWithH2DatabaseType() { .principalName(TEST_USER) .resourceType(TEST_RESOURCE_TYPE) .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) - .additionalParameters(TEST_JSON) + .additionalProperties(TEST_JSON) .build(); // Act @@ -136,7 +136,7 @@ public void testToMapWithH2DatabaseType() { assertEquals(TEST_USER, resultMap.get(PRINCIPAL_NAME)); assertEquals(TEST_RESOURCE_TYPE_STRING, resultMap.get(RESOURCE_TYPE)); assertEquals(TEST_RESOURCE_IDENTIFIER, resultMap.get(RESOURCE_IDENTIFIER)); - assertEquals(TEST_JSON, resultMap.get(ADDITIONAL_PARAMETERS)); + assertEquals(TEST_JSON, resultMap.get(ADDITIONAL_PROPERTIES)); } @Test @@ -152,7 +152,7 @@ public void testToMapWithPostgresType() { .principalName(TEST_USER) .resourceType(TEST_RESOURCE_TYPE) .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) - .additionalParameters(TEST_JSON) + .additionalProperties(TEST_JSON) .build(); // Act @@ -168,8 +168,8 @@ public void testToMapWithPostgresType() { assertEquals(TEST_RESOURCE_TYPE_STRING, resultMap.get(RESOURCE_TYPE)); assertEquals(TEST_RESOURCE_IDENTIFIER, resultMap.get(RESOURCE_IDENTIFIER)); - // For PostgreSQL, the additional parameters should be a PGobject of type "jsonb" - PGobject pgObject = (PGobject) resultMap.get(ADDITIONAL_PARAMETERS); + // For PostgreSQL, the additional properties should be a PGobject of type "jsonb" + PGobject pgObject = (PGobject) resultMap.get(ADDITIONAL_PROPERTIES); assertEquals("jsonb", pgObject.getType()); assertEquals(TEST_JSON, pgObject.getValue()); } @@ -196,7 +196,7 @@ public void testFromEvent() { TEST_USER, TEST_RESOURCE_TYPE, TEST_RESOURCE_IDENTIFIER); - polarisEvent.setAdditionalParameters(TEST_JSON); + polarisEvent.setAdditionalProperties(TEST_JSON); // Act ModelEvent result = ModelEvent.fromEvent(polarisEvent); @@ -210,7 +210,7 @@ public void testFromEvent() { assertEquals(TEST_USER, result.getPrincipalName()); assertEquals(TEST_RESOURCE_TYPE, result.getResourceType()); assertEquals(TEST_RESOURCE_IDENTIFIER, result.getResourceIdentifier()); - assertEquals(TEST_JSON, result.getAdditionalParameters()); + assertEquals(TEST_JSON, result.getAdditionalProperties()); } @Test @@ -235,7 +235,7 @@ public void testToEvent() { .principalName(TEST_USER) .resourceType(TEST_RESOURCE_TYPE) .resourceIdentifier(TEST_RESOURCE_IDENTIFIER) - .additionalParameters(TEST_JSON) + .additionalProperties(TEST_JSON) .build(); // Act @@ -250,6 +250,6 @@ public void testToEvent() { assertEquals(TEST_USER, result.getPrincipalName()); assertEquals(TEST_RESOURCE_TYPE, result.getResourceType()); assertEquals(TEST_RESOURCE_IDENTIFIER, result.getResourceIdentifier()); - assertEquals(TEST_JSON, result.getAdditionalParameters()); + assertEquals(TEST_JSON, result.getAdditionalProperties()); } } diff --git a/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java b/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java index fedb03a979..25b7e26b25 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java @@ -46,34 +46,22 @@ public class PolarisCallContext implements CallContext { private final Clock clock; - // A request ID to identify this REST request - private final String requestId; - private final RealmContext realmContext; private final RealmConfig realmConfig; - private static final String REQUEST_ID_KEY = "requestId"; - public PolarisCallContext( @Nonnull RealmContext realmContext, @Nonnull BasePersistence metaStore, @Nonnull PolarisDiagnostics diagServices, @Nonnull PolarisConfigurationStore configurationStore, - @Nonnull Clock clock, - ContainerRequestContext containerRequestContext) { + @Nonnull Clock clock) { this.realmContext = realmContext; this.metaStore = metaStore; this.diagServices = diagServices; this.configurationStore = configurationStore; this.clock = clock; this.realmConfig = new RealmConfigImpl(this.configurationStore, this.realmContext); - - String requestId = null; - if (containerRequestContext != null) { - requestId = (String) containerRequestContext.getProperty(REQUEST_ID_KEY); - } - this.requestId = requestId != null ? requestId : UUID.randomUUID().toString(); } public PolarisCallContext( @@ -85,8 +73,7 @@ public PolarisCallContext( metaStore, diagServices, new PolarisConfigurationStore() {}, - Clock.system(ZoneId.systemDefault()), - null); + Clock.system(ZoneId.systemDefault())); } public BasePersistence getMetaStore() { @@ -101,10 +88,6 @@ public Clock getClock() { return clock; } - public String getRequestId() { - return requestId; - } - @Override public RealmContext getRealmContext() { return realmContext; @@ -130,6 +113,6 @@ public PolarisCallContext copy() { String realmId = this.realmContext.getRealmIdentifier(); RealmContext realmContext = () -> realmId; return new PolarisCallContext( - realmContext, this.metaStore, this.diagServices, this.configurationStore, this.clock, null); + realmContext, this.metaStore, this.diagServices, this.configurationStore, this.clock); } } diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java index 7e616de139..ca80df7795 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java @@ -55,7 +55,7 @@ public class PolarisEvent { private String resourceIdentifier; // Additional parameters that were not earlier recorded - private String additionalParameters; + private String additionalProperties; public String getCatalogId() { return catalogId; @@ -89,8 +89,8 @@ public String getResourceIdentifier() { return resourceIdentifier; } - public String getAdditionalParameters() { - return additionalParameters != null ? additionalParameters : EMPTY_MAP_STRING; + public String getAdditionalProperties() { + return additionalProperties != null ? additionalProperties : EMPTY_MAP_STRING; } public PolarisEvent( @@ -113,17 +113,17 @@ public PolarisEvent( } @JsonIgnore - public void setAdditionalParameters(Map properties) { + public void setAdditionalProperties(Map properties) { try { - this.additionalParameters = properties == null ? null : MAPPER.writeValueAsString(properties); + this.additionalProperties = properties == null ? null : MAPPER.writeValueAsString(properties); } catch (JsonProcessingException ex) { throw new IllegalStateException( String.format("Failed to serialize json. properties %s", properties), ex); } } - public void setAdditionalParameters(String additionalParameters) { - this.additionalParameters = additionalParameters; + public void setAdditionalProperties(String additionalProperties) { + this.additionalProperties = additionalProperties; } public enum ResourceType { diff --git a/runtime/defaults/src/main/resources/application.properties b/runtime/defaults/src/main/resources/application.properties index 71fe2798d1..53a00b7d9e 100644 --- a/runtime/defaults/src/main/resources/application.properties +++ b/runtime/defaults/src/main/resources/application.properties @@ -109,10 +109,11 @@ polaris.realm-context.header-name=Polaris-Realm polaris.realm-context.require-header=false polaris.features."ENFORCE_PRINCIPAL_CREDENTIAL_ROTATION_REQUIRED_CHECKING"=false -polaris.features."SUPPORTED_CATALOG_STORAGE_TYPES"=["S3","GCS","AZURE"] +polaris.features."SUPPORTED_CATALOG_STORAGE_TYPES"=["S3","GCS","AZURE","FILE"] +polaris.features."ALLOW_INSECURE_STORAGE_TYPES"=true +polaris.readiness.ignore-severe-issues=true # polaris.features."ENABLE_CATALOG_FEDERATION"=true polaris.features."SUPPORTED_CATALOG_CONNECTION_TYPES"=["ICEBERG_REST"] -polaris.features."SUPPORTED_EXTERNAL_CATALOG_AUTHENTICATION_TYPES"=["OAUTH", "BEARER"] # realm overrides # polaris.features.realm-overrides."my-realm"."SKIP_CREDENTIAL_SUBSCOPING_INDIRECTION"=true @@ -128,8 +129,8 @@ polaris.file-io.type=default polaris.event-listener.type=no-op # polaris.event-listener.type=persistence-in-memory-buffer -# polaris.event-listener.buffer-time=5000ms -# polaris.event-listener.max-buffer-size=5 +# polaris.event-listener.persistence-in-memory-buffer.buffer-time=5000ms +# polaris.event-listener.persistence-in-memory-buffer.max-buffer-size=5 polaris.log.request-id-header-name=Polaris-Request-Id diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java index 5f35a5efeb..bbe5b180dd 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java @@ -131,8 +131,7 @@ public CallContext polarisCallContext( PolarisDiagnostics diagServices, PolarisConfigurationStore configurationStore, MetaStoreManagerFactory metaStoreManagerFactory, - Clock clock, - ContainerRequestContext containerRequestContext) { + Clock clock) { BasePersistence metaStoreSession = metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(); return new PolarisCallContext( @@ -140,8 +139,7 @@ public CallContext polarisCallContext( metaStoreSession, diagServices, configurationStore, - clock, - containerRequestContext); + clock); } @Produces diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java index 2a5452ceb7..2497ab55ae 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisEventListenerConfiguration.java @@ -20,23 +20,14 @@ import io.quarkus.runtime.annotations.StaticInitSafe; import io.smallrye.config.ConfigMapping; -import java.time.Duration; -import java.util.Optional; -import org.apache.polaris.service.events.EventListenerConfiguration; import org.apache.polaris.service.events.listeners.PolarisEventListener; @StaticInitSafe @ConfigMapping(prefix = "polaris.event-listener") -public interface QuarkusPolarisEventListenerConfiguration extends EventListenerConfiguration { +public interface QuarkusPolarisEventListenerConfiguration { /** * The type of the event listener to use. Must be a registered {@link PolarisEventListener} * identifier. */ String type(); - - @Override - Optional bufferTime(); - - @Override - Optional maxBufferSize(); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java new file mode 100644 index 0000000000..bd44b0bb24 --- /dev/null +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java @@ -0,0 +1,51 @@ +/* + * 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.polaris.service.quarkus.events; + +import io.quarkus.runtime.annotations.StaticInitSafe; +import io.smallrye.config.ConfigMapping; +import io.smallrye.config.WithDefault; +import io.smallrye.config.WithName; +import jakarta.enterprise.context.ApplicationScoped; +import org.apache.polaris.service.events.listeners.InMemoryBufferPersistenceListenerConfiguration; + +import java.time.Duration; +import java.util.Optional; + +@StaticInitSafe +@ConfigMapping(prefix = "polaris.event-listener.persistence-in-memory-buffer") +@ApplicationScoped +public interface QuarkusPolarisInMemoryBufferEventListenerConfiguration extends InMemoryBufferPersistenceListenerConfiguration { + /** + * @return the buffer time in milliseconds + */ + @Override + @WithName("buffer-time") + @WithDefault("5000ms") + Optional bufferTime(); + + /** + * @return the maximum number of cached entries + */ + @Override + @WithName("max-buffer-size") + @WithDefault("5") + Optional maxBufferSize(); +} diff --git a/service/common/src/main/java/org/apache/polaris/service/context/DefaultCallContextResolver.java b/service/common/src/main/java/org/apache/polaris/service/context/DefaultCallContextResolver.java index 6af2b00af0..62b113daea 100644 --- a/service/common/src/main/java/org/apache/polaris/service/context/DefaultCallContextResolver.java +++ b/service/common/src/main/java/org/apache/polaris/service/context/DefaultCallContextResolver.java @@ -67,6 +67,6 @@ public CallContext resolveCallContext( BasePersistence metaStoreSession = metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(); return new PolarisCallContext( - realmContext, metaStoreSession, diagnostics, configurationStore, clock, null); + realmContext, metaStoreSession, diagnostics, configurationStore, clock); } } diff --git a/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPersistenceListenerConfiguration.java similarity index 91% rename from service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java rename to service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPersistenceListenerConfiguration.java index c3ca6cb0b5..34ea164e18 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/EventListenerConfiguration.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPersistenceListenerConfiguration.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.polaris.service.events; +package org.apache.polaris.service.events.listeners; import java.time.Duration; import java.util.Optional; @@ -29,7 +29,7 @@ * *

maxBufferSize() specifies the maximum number of cached entries. */ -public interface EventListenerConfiguration { +public interface InMemoryBufferPersistenceListenerConfiguration { /** * @return the buffer time in milliseconds */ diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java index 5deac1c5c8..d9454a890d 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java @@ -28,17 +28,20 @@ import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; + +import jakarta.ws.rs.container.ContainerRequestContext; +import jakarta.ws.rs.core.Context; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.persistence.MetaStoreManagerFactory; -import org.apache.polaris.service.events.EventListenerConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,6 +51,7 @@ public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersistenceEventListener { private static final Logger LOGGER = LoggerFactory.getLogger(InMemoryBufferPolarisPersistenceEventListener.class); + private static final String REQUEST_ID_KEY = "requestId"; private final MetaStoreManagerFactory metaStoreManagerFactory; private final Clock clock; @@ -58,13 +62,16 @@ public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersis private final Duration timeToFlush; private final int maxBufferSize; + @Context + ContainerRequestContext containerRequestContext; + private record EventAndContext(PolarisEvent polarisEvent, PolarisCallContext callContext) {} @Inject public InMemoryBufferPolarisPersistenceEventListener( MetaStoreManagerFactory metaStoreManagerFactory, Clock clock, - EventListenerConfiguration eventListenerConfiguration) { + InMemoryBufferPersistenceListenerConfiguration eventListenerConfiguration) { this.metaStoreManagerFactory = metaStoreManagerFactory; this.clock = clock; this.timeToFlush = @@ -102,6 +109,14 @@ void shutdown() { executor.shutdownNow(); } + @Override + String getRequestId() { + if (containerRequestContext != null && containerRequestContext.hasProperty(REQUEST_ID_KEY)) { + return (String) containerRequestContext.getProperty(REQUEST_ID_KEY); + } + return UUID.randomUUID().toString(); + } + @Override void addToBuffer(PolarisEvent polarisEvent, CallContext callCtx) { String realmId = callCtx.getRealmContext().getRealmIdentifier(); diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java index dda314bc36..908fd9b7fa 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java @@ -88,7 +88,7 @@ public void onAfterTableCreated( new org.apache.polaris.core.entity.PolarisEvent( event.catalogName(), event.eventId(), - getRequestId(callCtx), + getRequestId(), event.getClass().getSimpleName(), getTimestamp(callCtx), getUsername(securityContext), @@ -100,7 +100,7 @@ public void onAfterTableCreated( event.metadata().uuid(), "metadata", TableMetadataParser.toJson(event.metadata())); - polarisEvent.setAdditionalParameters(additionalParameters); + polarisEvent.setAdditionalProperties(additionalParameters); addToBuffer(polarisEvent, callCtx); } @@ -112,7 +112,7 @@ public void onAfterCatalogCreated( new PolarisEvent( event.catalogName(), event.eventId(), - getRequestId(callCtx), + getRequestId(), event.getClass().getSimpleName(), getTimestamp(callCtx), getUsername(securityContext), @@ -125,9 +125,7 @@ private long getTimestamp(CallContext callCtx) { return callCtx.getPolarisCallContext().getClock().millis(); } - private String getRequestId(CallContext callCtx) { - return callCtx.getPolarisCallContext().getRequestId(); - } + abstract String getRequestId(); private String getUsername(SecurityContext securityContext) { return securityContext.getUserPrincipal() == null diff --git a/service/common/src/test/java/org/apache/polaris/service/catalog/io/FileIOFactoryTest.java b/service/common/src/test/java/org/apache/polaris/service/catalog/io/FileIOFactoryTest.java index 1ccd6a0c09..34a9531d23 100644 --- a/service/common/src/test/java/org/apache/polaris/service/catalog/io/FileIOFactoryTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/catalog/io/FileIOFactoryTest.java @@ -142,8 +142,7 @@ FileIO loadFileIOInternal( testServices.metaStoreManagerFactory().getOrCreateSessionSupplier(realmContext).get(), testServices.polarisDiagnostics(), testServices.configurationStore(), - Clock.systemUTC(), - null); + Clock.systemUTC()); } @AfterEach diff --git a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index ec570deeea..b047339d99 100644 --- a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -19,6 +19,7 @@ package org.apache.polaris.service.events.listeners; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -34,13 +35,13 @@ import java.util.Optional; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; +import jakarta.ws.rs.container.ContainerRequestContext; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.context.RealmContext; import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.persistence.MetaStoreManagerFactory; import org.apache.polaris.core.persistence.PolarisMetaStoreManager; -import org.apache.polaris.service.events.EventListenerConfiguration; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -67,8 +68,8 @@ public void setUp() { when(metaStoreManagerFactory.getOrCreateMetaStoreManager(Mockito.any())) .thenReturn(polarisMetaStoreManager); - EventListenerConfiguration eventListenerConfiguration = - Mockito.mock(EventListenerConfiguration.class); + InMemoryBufferPersistenceListenerConfiguration eventListenerConfiguration = + Mockito.mock(InMemoryBufferPersistenceListenerConfiguration.class); when(eventListenerConfiguration.maxBufferSize()) .thenReturn(Optional.of(CONFIG_MAX_BUFFER_SIZE)); when(eventListenerConfiguration.bufferTime()) @@ -159,6 +160,86 @@ public void testCheckAndFlushBufferIfNecessaryIsThreadSafe() throws Exception { .writeEvents(eq(callContext.getPolarisCallContext()), eq(events)); } + @Test + public void testRequestIdFunctionalityWithContainerRequestContext() { + // Test when containerRequestContext has requestId property + ContainerRequestContext mockContainerRequestContext = Mockito.mock(ContainerRequestContext.class); + String expectedRequestId = "custom-request-id-123"; + + when(mockContainerRequestContext.hasProperty("requestId")).thenReturn(true); + when(mockContainerRequestContext.getProperty("requestId")).thenReturn(expectedRequestId); + + // Use reflection to set the containerRequestContext field + try { + java.lang.reflect.Field field = + InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField("containerRequestContext"); + field.setAccessible(true); + field.set(eventListener, mockContainerRequestContext); + } catch (Exception e) { + throw new RuntimeException("Failed to set containerRequestContext field", e); + } + + String actualRequestId = eventListener.getRequestId(); + assertThat(actualRequestId) + .as("Expected requestId '" + expectedRequestId + "' but got '" + actualRequestId + "'") + .isEqualTo(expectedRequestId); + } + + @Test + public void testRequestIdFunctionalityWithoutContainerRequestContext() { + // Test when containerRequestContext is null + try { + java.lang.reflect.Field field = + InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField("containerRequestContext"); + field.setAccessible(true); + field.set(eventListener, null); + } catch (Exception e) { + throw new RuntimeException("Failed to set containerRequestContext field", e); + } + + String requestId1 = eventListener.getRequestId(); + String requestId2 = eventListener.getRequestId(); + + // Both should be valid UUIDs + assertThat(isValidUUID(requestId1)) + .as("Generated requestId should be a valid UUID: " + requestId1) + .isTrue(); + assertThat(isValidUUID(requestId2)) + .as("Generated requestId should be a valid UUID: " + requestId2) + .isTrue(); + + // Each call should generate a different UUID + assertThat(requestId1) + .as("Each call to getRequestId() should generate a different UUID") + .isNotEqualTo(requestId2); + } + + @Test + public void testRequestIdFunctionalityWithContainerRequestContextButNoProperty() { + // Test when containerRequestContext exists but doesn't have requestId property + ContainerRequestContext mockContainerRequestContext = Mockito.mock(ContainerRequestContext.class); + when(mockContainerRequestContext.hasProperty("requestId")).thenReturn(false); + + try { + java.lang.reflect.Field field = + InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField("containerRequestContext"); + field.setAccessible(true); + field.set(eventListener, mockContainerRequestContext); + } catch (Exception e) { + throw new RuntimeException("Failed to set containerRequestContext field", e); + } + + String requestId = eventListener.getRequestId(); + + // Should generate a UUID since property is not available + assertThat(isValidUUID(requestId)) + .as("Generated requestId should be a valid UUID: " + requestId) + .isTrue(); + + // Verify that getProperty was never called since hasProperty returned false + verify(mockContainerRequestContext, times(0)).getProperty("requestId"); + } + private List addEventsWithoutTriggeringFlush(String realmId) { List realmEvents = new ArrayList<>(); for (int i = 0; i < CONFIG_MAX_BUFFER_SIZE - 1; i++) { @@ -196,8 +277,17 @@ private PolarisEvent createSampleEvent() { Map additionalParams = new HashMap<>(); additionalParams.put("testKey", "testValue"); - event.setAdditionalParameters(additionalParams); + event.setAdditionalProperties(additionalParams); return event; } + + private boolean isValidUUID(String uuid) { + try { + UUID.fromString(uuid); + return true; + } catch (IllegalArgumentException e) { + return false; + } + } } diff --git a/service/common/src/testFixtures/java/org/apache/polaris/service/TestServices.java b/service/common/src/testFixtures/java/org/apache/polaris/service/TestServices.java index 28053ed7ff..1fed3d863a 100644 --- a/service/common/src/testFixtures/java/org/apache/polaris/service/TestServices.java +++ b/service/common/src/testFixtures/java/org/apache/polaris/service/TestServices.java @@ -165,8 +165,7 @@ public TestServices build() { metaStoreSession, polarisDiagnostics, configurationStore, - Clock.systemUTC(), - null); + Clock.systemUTC()); PolarisEntityManager entityManager = realmEntityManagerFactory.getOrCreateEntityManager(realmContext); PolarisMetaStoreManager metaStoreManager = From de088e4faddf515204cdbe0901ba40c7b7abacdc Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Wed, 23 Jul 2025 08:05:58 -0700 Subject: [PATCH 06/17] merge conflicts --- ...olarisEclipseLinkMetaStoreManagerTest.java | 3 +- ...anagerWithJdbcBasePersistenceImplTest.java | 3 +- .../polaris/core/PolarisCallContext.java | 2 -- ...apAtomicOperationMetaStoreManagerTest.java | 3 +- .../PolarisTreeMapMetaStoreManagerTest.java | 3 +- .../InMemoryStorageIntegrationTest.java | 3 +- .../quarkus/config/QuarkusProducers.java | 6 +--- ...emoryBufferEventListenerConfiguration.java | 34 +++++++++---------- .../quarkus/admin/ManagementServiceTest.java | 3 +- .../quarkus/auth/JWTRSAKeyPairTest.java | 2 +- .../auth/JWTSymmetricKeyGeneratorTest.java | 3 +- .../catalog/AbstractIcebergCatalogTest.java | 3 +- .../AbstractIcebergCatalogViewTest.java | 3 +- ...bstractPolarisGenericTableCatalogTest.java | 3 +- .../catalog/AbstractPolicyCatalogTest.java | 4 +-- .../task/TableCleanupTaskHandlerTest.java | 3 +- .../test/PolarisIntegrationTestFixture.java | 3 +- ...BufferPolarisPersistenceEventListener.java | 8 ++--- ...erPolarisPersistenceEventListenerTest.java | 17 ++++++---- 19 files changed, 46 insertions(+), 63 deletions(-) diff --git a/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java b/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java index 2e7328884e..55607981e9 100644 --- a/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java +++ b/persistence/eclipselink/src/test/java/org/apache/polaris/extension/persistence/impl/eclipselink/PolarisEclipseLinkMetaStoreManagerTest.java @@ -96,8 +96,7 @@ protected PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { session, diagServices, new PolarisConfigurationStore() {}, - timeSource.withZone(ZoneId.systemDefault()), - null)); + timeSource.withZone(ZoneId.systemDefault()))); } @ParameterizedTest diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/AtomicMetastoreManagerWithJdbcBasePersistenceImplTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/AtomicMetastoreManagerWithJdbcBasePersistenceImplTest.java index be5cfceef9..bbb57e68ba 100644 --- a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/AtomicMetastoreManagerWithJdbcBasePersistenceImplTest.java +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/AtomicMetastoreManagerWithJdbcBasePersistenceImplTest.java @@ -76,8 +76,7 @@ protected PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { basePersistence, diagServices, new PolarisConfigurationStore() {}, - timeSource.withZone(ZoneId.systemDefault()), - null)); + timeSource.withZone(ZoneId.systemDefault()))); } private static class H2JdbcConfiguration implements RelationalJdbcConfiguration { diff --git a/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java b/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java index 25b7e26b25..cf2a0cca4c 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/PolarisCallContext.java @@ -19,10 +19,8 @@ package org.apache.polaris.core; import jakarta.annotation.Nonnull; -import jakarta.ws.rs.container.ContainerRequestContext; import java.time.Clock; import java.time.ZoneId; -import java.util.UUID; import org.apache.polaris.core.config.PolarisConfigurationStore; import org.apache.polaris.core.config.RealmConfig; import org.apache.polaris.core.config.RealmConfigImpl; diff --git a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java index 6c65a2ef48..f89615cf10 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapAtomicOperationMetaStoreManagerTest.java @@ -41,8 +41,7 @@ public PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { new TreeMapTransactionalPersistenceImpl(store, Mockito.mock(), RANDOM_SECRETS), diagServices, new PolarisConfigurationStore() {}, - timeSource.withZone(ZoneId.systemDefault()), - null); + timeSource.withZone(ZoneId.systemDefault())); return new PolarisTestMetaStoreManager(new AtomicOperationMetaStoreManager(), callCtx); } diff --git a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java index 462c4b58a3..49a2bfcc00 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/persistence/PolarisTreeMapMetaStoreManagerTest.java @@ -41,8 +41,7 @@ public PolarisTestMetaStoreManager createPolarisTestMetaStoreManager() { new TreeMapTransactionalPersistenceImpl(store, Mockito.mock(), RANDOM_SECRETS), diagServices, new PolarisConfigurationStore() {}, - timeSource.withZone(ZoneId.systemDefault()), - null); + timeSource.withZone(ZoneId.systemDefault())); return new PolarisTestMetaStoreManager(new TransactionalMetaStoreManagerImpl(), callCtx); } diff --git a/polaris-core/src/test/java/org/apache/polaris/core/storage/InMemoryStorageIntegrationTest.java b/polaris-core/src/test/java/org/apache/polaris/core/storage/InMemoryStorageIntegrationTest.java index 3e847b434a..d0bf9de8a9 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/storage/InMemoryStorageIntegrationTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/storage/InMemoryStorageIntegrationTest.java @@ -111,8 +111,7 @@ public void testValidateAccessToLocationsWithWildcard(String s3Scheme) { return (T) config.get(configName); } }, - Clock.systemUTC(), - null); + Clock.systemUTC()); CallContext.setCurrentContext(polarisCallContext); Map> result = storage.validateAccessToLocations( diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java index 74803fcce5..c1d2b55b8d 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java @@ -137,11 +137,7 @@ public CallContext polarisCallContext( BasePersistence metaStoreSession = metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(); return new PolarisCallContext( - realmContext, - metaStoreSession, - diagServices, - configurationStore, - clock); + realmContext, metaStoreSession, diagServices, configurationStore, clock); } @Produces diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java index bd44b0bb24..3d51e8b9c2 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java @@ -24,28 +24,28 @@ import io.smallrye.config.WithDefault; import io.smallrye.config.WithName; import jakarta.enterprise.context.ApplicationScoped; -import org.apache.polaris.service.events.listeners.InMemoryBufferPersistenceListenerConfiguration; - import java.time.Duration; import java.util.Optional; +import org.apache.polaris.service.events.listeners.InMemoryBufferPersistenceListenerConfiguration; @StaticInitSafe @ConfigMapping(prefix = "polaris.event-listener.persistence-in-memory-buffer") @ApplicationScoped -public interface QuarkusPolarisInMemoryBufferEventListenerConfiguration extends InMemoryBufferPersistenceListenerConfiguration { - /** - * @return the buffer time in milliseconds - */ - @Override - @WithName("buffer-time") - @WithDefault("5000ms") - Optional bufferTime(); +public interface QuarkusPolarisInMemoryBufferEventListenerConfiguration + extends InMemoryBufferPersistenceListenerConfiguration { + /** + * @return the buffer time in milliseconds + */ + @Override + @WithName("buffer-time") + @WithDefault("5000ms") + Optional bufferTime(); - /** - * @return the maximum number of cached entries - */ - @Override - @WithName("max-buffer-size") - @WithDefault("5") - Optional maxBufferSize(); + /** + * @return the maximum number of cached entries + */ + @Override + @WithName("max-buffer-size") + @WithDefault("5") + Optional maxBufferSize(); } diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java index b9aa7cf758..319a0e354c 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/admin/ManagementServiceTest.java @@ -85,8 +85,7 @@ public void setup() { .get(), fakeServices.polarisDiagnostics(), fakeServices.configurationStore(), - Mockito.mock(Clock.class), - null); + Mockito.mock(Clock.class)); CallContext.setCurrentContext(polarisCallContext); services = TestServices.builder() diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTRSAKeyPairTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTRSAKeyPairTest.java index b633050ef8..46d2950b91 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTRSAKeyPairTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTRSAKeyPairTest.java @@ -61,7 +61,7 @@ public void testSuccessfulTokenGeneration() throws Exception { final String scope = "PRINCIPAL_ROLE:TEST"; PolarisCallContext polarisCallContext = - new PolarisCallContext(null, null, null, configurationStore, null, null); + new PolarisCallContext(null, null, null, configurationStore, null); PolarisMetaStoreManager metastoreManager = Mockito.mock(PolarisMetaStoreManager.class); String mainSecret = "client-secret"; PolarisPrincipalSecrets principalSecrets = diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java index 9a6c40c8f5..6b51f6eda3 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/auth/JWTSymmetricKeyGeneratorTest.java @@ -45,8 +45,7 @@ public class JWTSymmetricKeyGeneratorTest { /** Sanity test to verify that we can generate a token */ @Test public void testJWTSymmetricKeyGenerator() { - PolarisCallContext polarisCallContext = - new PolarisCallContext(null, null, null, null, null, null); + PolarisCallContext polarisCallContext = new PolarisCallContext(null, null, null, null, null); PolarisMetaStoreManager metastoreManager = Mockito.mock(PolarisMetaStoreManager.class); String mainSecret = "test_secret"; String clientId = "test_client_id"; diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractIcebergCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractIcebergCatalogTest.java index 71faa7932f..e39d8655ec 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractIcebergCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractIcebergCatalogTest.java @@ -277,8 +277,7 @@ public void before(TestInfo testInfo) { metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - Clock.systemDefaultZone(), - null); + Clock.systemDefaultZone()); EntityCache entityCache = createEntityCache(polarisContext.getRealmConfig(), metaStoreManager); entityManager = new PolarisEntityManager(metaStoreManager, entityCache); diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractIcebergCatalogViewTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractIcebergCatalogViewTest.java index 6bed5333c5..15c64f2dea 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractIcebergCatalogViewTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractIcebergCatalogViewTest.java @@ -164,8 +164,7 @@ public void before(TestInfo testInfo) { metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - Clock.systemDefaultZone(), - null); + Clock.systemDefaultZone()); EntityCache entityCache = metaStoreManagerFactory.getOrCreateEntityCache( diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolarisGenericTableCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolarisGenericTableCatalogTest.java index e8e50ecc75..bf87da3352 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolarisGenericTableCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolarisGenericTableCatalogTest.java @@ -57,6 +57,7 @@ import org.apache.polaris.core.persistence.PolarisEntityManager; import org.apache.polaris.core.persistence.PolarisMetaStoreManager; import org.apache.polaris.core.persistence.cache.EntityCache; +import org.apache.polaris.core.persistence.cache.InMemoryEntityCache; import org.apache.polaris.core.secrets.UserSecretsManager; import org.apache.polaris.core.secrets.UserSecretsManagerFactory; import org.apache.polaris.core.storage.PolarisStorageIntegration; @@ -159,9 +160,7 @@ public void before(TestInfo testInfo) { entityManager = new PolarisEntityManager( metaStoreManager, - new StorageCredentialCache(), new InMemoryEntityCache(polarisContext.getRealmConfig(), metaStoreManager)); - Clock.systemDefaultZone()); EntityCache entityCache = metaStoreManagerFactory.getOrCreateEntityCache( diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolicyCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolicyCatalogTest.java index a79c63d81d..a701f1b3a9 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolicyCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolicyCatalogTest.java @@ -65,6 +65,7 @@ import org.apache.polaris.core.persistence.PolarisMetaStoreManager; import org.apache.polaris.core.persistence.PolicyMappingAlreadyExistsException; import org.apache.polaris.core.persistence.cache.EntityCache; +import org.apache.polaris.core.persistence.cache.InMemoryEntityCache; import org.apache.polaris.core.policy.PredefinedPolicyTypes; import org.apache.polaris.core.policy.exceptions.NoSuchPolicyException; import org.apache.polaris.core.policy.exceptions.PolicyInUseException; @@ -182,15 +183,12 @@ public void before(TestInfo testInfo) { entityManager = new PolarisEntityManager( metaStoreManager, - new StorageCredentialCache(), new InMemoryEntityCache(polarisContext.getRealmConfig(), metaStoreManager)); - Clock.systemDefaultZone()); EntityCache entityCache = metaStoreManagerFactory.getOrCreateEntityCache( realmContext, polarisContext.getRealmConfig()); entityManager = new PolarisEntityManager(metaStoreManager, entityCache); ->>>>>>> origin/main:runtime/service/src/test/java/org/apache/polaris/service/quarkus/catalog/AbstractPolicyCatalogTest.java callContext = polarisContext; diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java index 1b4bf83529..69304956b0 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java @@ -87,8 +87,7 @@ void setup() { metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), diagServices, configurationStore, - Clock.systemDefaultZone(), - null); + Clock.systemDefaultZone()); } @Test diff --git a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java index a33e43fdbb..e9268befb7 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/quarkus/test/PolarisIntegrationTestFixture.java @@ -118,8 +118,7 @@ private PolarisPrincipalSecrets fetchAdminSecrets() { metaStoreSession, helper.diagServices, helper.configurationStore, - helper.clock, - null); + helper.clock); try { PolarisMetaStoreManager metaStoreManager = helper.metaStoreManagerFactory.getOrCreateMetaStoreManager(realmContext); diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java index d9454a890d..c217745244 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java @@ -24,6 +24,8 @@ import jakarta.annotation.PreDestroy; import jakarta.enterprise.context.ApplicationScoped; import jakarta.inject.Inject; +import jakarta.ws.rs.container.ContainerRequestContext; +import jakarta.ws.rs.core.Context; import java.time.Clock; import java.time.Duration; import java.time.temporal.ChronoUnit; @@ -35,9 +37,6 @@ import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - -import jakarta.ws.rs.container.ContainerRequestContext; -import jakarta.ws.rs.core.Context; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.entity.PolarisEvent; @@ -62,8 +61,7 @@ public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersis private final Duration timeToFlush; private final int maxBufferSize; - @Context - ContainerRequestContext containerRequestContext; + @Context ContainerRequestContext containerRequestContext; private record EventAndContext(PolarisEvent polarisEvent, PolarisCallContext callContext) {} diff --git a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index b047339d99..b76cb664dc 100644 --- a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import jakarta.ws.rs.container.ContainerRequestContext; import java.time.Duration; import java.time.Instant; import java.time.ZoneOffset; @@ -35,7 +36,6 @@ import java.util.Optional; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; -import jakarta.ws.rs.container.ContainerRequestContext; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.context.RealmContext; @@ -163,7 +163,8 @@ public void testCheckAndFlushBufferIfNecessaryIsThreadSafe() throws Exception { @Test public void testRequestIdFunctionalityWithContainerRequestContext() { // Test when containerRequestContext has requestId property - ContainerRequestContext mockContainerRequestContext = Mockito.mock(ContainerRequestContext.class); + ContainerRequestContext mockContainerRequestContext = + Mockito.mock(ContainerRequestContext.class); String expectedRequestId = "custom-request-id-123"; when(mockContainerRequestContext.hasProperty("requestId")).thenReturn(true); @@ -172,7 +173,8 @@ public void testRequestIdFunctionalityWithContainerRequestContext() { // Use reflection to set the containerRequestContext field try { java.lang.reflect.Field field = - InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField("containerRequestContext"); + InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField( + "containerRequestContext"); field.setAccessible(true); field.set(eventListener, mockContainerRequestContext); } catch (Exception e) { @@ -190,7 +192,8 @@ public void testRequestIdFunctionalityWithoutContainerRequestContext() { // Test when containerRequestContext is null try { java.lang.reflect.Field field = - InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField("containerRequestContext"); + InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField( + "containerRequestContext"); field.setAccessible(true); field.set(eventListener, null); } catch (Exception e) { @@ -217,12 +220,14 @@ public void testRequestIdFunctionalityWithoutContainerRequestContext() { @Test public void testRequestIdFunctionalityWithContainerRequestContextButNoProperty() { // Test when containerRequestContext exists but doesn't have requestId property - ContainerRequestContext mockContainerRequestContext = Mockito.mock(ContainerRequestContext.class); + ContainerRequestContext mockContainerRequestContext = + Mockito.mock(ContainerRequestContext.class); when(mockContainerRequestContext.hasProperty("requestId")).thenReturn(false); try { java.lang.reflect.Field field = - InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField("containerRequestContext"); + InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField( + "containerRequestContext"); field.setAccessible(true); field.set(eventListener, mockContainerRequestContext); } catch (Exception e) { From b2317e0ef47be0acd3e27984e9cbe1f6c73400fc Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Thu, 7 Aug 2025 14:31:49 -0700 Subject: [PATCH 07/17] addressed review from @snazy --- .../src/main/resources/application.properties | 5 +- ...emoryBufferEventListenerConfiguration.java | 3 +- .../service/admin/PolarisServiceImpl.java | 4 +- .../catalog/iceberg/IcebergCatalog.java | 33 ++------ .../iceberg/IcebergCatalogHandler.java | 8 +- ...BufferPolarisPersistenceEventListener.java | 83 +++++++++++-------- .../listeners/PolarisEventListener.java | 39 +++------ .../PolarisPersistenceEventListener.java | 82 +++++++----------- .../listeners/TestPolarisEventListener.java | 33 +++----- .../ratelimiter/RateLimiterFilter.java | 3 +- .../service/task/TaskExecutorImpl.java | 5 +- ...erPolarisPersistenceEventListenerTest.java | 20 +++-- 12 files changed, 136 insertions(+), 182 deletions(-) diff --git a/runtime/defaults/src/main/resources/application.properties b/runtime/defaults/src/main/resources/application.properties index 039b5d7180..f9f468698f 100644 --- a/runtime/defaults/src/main/resources/application.properties +++ b/runtime/defaults/src/main/resources/application.properties @@ -109,9 +109,7 @@ polaris.realm-context.header-name=Polaris-Realm polaris.realm-context.require-header=false polaris.features."ENFORCE_PRINCIPAL_CREDENTIAL_ROTATION_REQUIRED_CHECKING"=false -polaris.features."SUPPORTED_CATALOG_STORAGE_TYPES"=["S3","GCS","AZURE","FILE"] -polaris.features."ALLOW_INSECURE_STORAGE_TYPES"=true -polaris.readiness.ignore-severe-issues=true +polaris.features."SUPPORTED_CATALOG_STORAGE_TYPES"=["S3","GCS","AZURE"] # polaris.features."ENABLE_CATALOG_FEDERATION"=true polaris.features."SUPPORTED_CATALOG_CONNECTION_TYPES"=["ICEBERG_REST"] @@ -132,7 +130,6 @@ polaris.event-listener.type=no-op # polaris.event-listener.persistence-in-memory-buffer.buffer-time=5000ms # polaris.event-listener.persistence-in-memory-buffer.max-buffer-size=5 - polaris.log.request-id-header-name=Polaris-Request-Id # polaris.log.mdc.aid=polaris # polaris.log.mdc.sid=polaris-service diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java index 3d51e8b9c2..6ddec631eb 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java @@ -19,16 +19,15 @@ package org.apache.polaris.service.quarkus.events; -import io.quarkus.runtime.annotations.StaticInitSafe; import io.smallrye.config.ConfigMapping; import io.smallrye.config.WithDefault; import io.smallrye.config.WithName; import jakarta.enterprise.context.ApplicationScoped; import java.time.Duration; import java.util.Optional; + import org.apache.polaris.service.events.listeners.InMemoryBufferPersistenceListenerConfiguration; -@StaticInitSafe @ConfigMapping(prefix = "polaris.event-listener.persistence-in-memory-buffer") @ApplicationScoped public interface QuarkusPolarisInMemoryBufferEventListenerConfiguration diff --git a/service/common/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java b/service/common/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java index f54d90a656..10a5313d67 100644 --- a/service/common/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java +++ b/service/common/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java @@ -151,9 +151,7 @@ public Response createCatalog( Catalog newCatalog = new CatalogEntity(adminService.createCatalog(request)).asCatalog(); LOGGER.info("Created new catalog {}", newCatalog); polarisEventListener.onAfterCatalogCreated( - new AfterCatalogCreatedEvent(PolarisEvent.createEventId(), newCatalog.getName()), - callContext, - securityContext); + new AfterCatalogCreatedEvent(PolarisEvent.createEventId(), newCatalog.getName())); return Response.status(Response.Status.CREATED).build(); } diff --git a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java index 348ffe3656..6647ff5d98 100644 --- a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java +++ b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java @@ -1422,9 +1422,7 @@ public void doRefresh() { disableRefresh(); } else { polarisEventListener.onBeforeTableRefreshed( - new BeforeTableRefreshedEvent(PolarisEvent.createEventId(), tableIdentifier), - callContext, - securityContext); + new BeforeTableRefreshedEvent(PolarisEvent.createEventId(), tableIdentifier)); refreshFromMetadataLocation( latestLocation, SHOULD_RETRY_REFRESH_PREDICATE, @@ -1445,18 +1443,13 @@ public void doRefresh() { return TableMetadataParser.read(fileIO, metadataLocation); }); polarisEventListener.onAfterTableRefreshed( - new AfterTableRefreshedEvent(PolarisEvent.createEventId(), tableIdentifier), - callContext, - securityContext); + new AfterTableRefreshedEvent(PolarisEvent.createEventId(), tableIdentifier)); } } public void doCommit(TableMetadata base, TableMetadata metadata) { polarisEventListener.onBeforeTableCommited( - new BeforeTableCommitedEvent( - PolarisEvent.createEventId(), tableIdentifier, base, metadata), - callContext, - securityContext); + new BeforeTableCommitedEvent(PolarisEvent.createEventId(), tableIdentifier, base, metadata)); LOGGER.debug( "doCommit for table {} with base {}, metadata {}", tableIdentifier, base, metadata); @@ -1615,9 +1608,7 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { polarisEventListener.onAfterTableCommited( new AfterTableCommitedEvent( - PolarisEvent.createEventId(), tableIdentifier, base, metadata), - callContext, - securityContext); + PolarisEvent.createEventId(), tableIdentifier, base, metadata)); } @Override @@ -1809,9 +1800,7 @@ public void doRefresh() { disableRefresh(); } else { polarisEventListener.onBeforeViewRefreshed( - new BeforeViewRefreshedEvent(PolarisEvent.createEventId(), identifier), - callContext, - securityContext); + new BeforeViewRefreshedEvent(PolarisEvent.createEventId(), identifier)); refreshFromMetadataLocation( latestLocation, SHOULD_RETRY_REFRESH_PREDICATE, @@ -1834,17 +1823,13 @@ public void doRefresh() { return ViewMetadataParser.read(fileIO.newInputFile(metadataLocation)); }); polarisEventListener.onAfterViewRefreshed( - new AfterViewRefreshedEvent(PolarisEvent.createEventId(), identifier), - callContext, - securityContext); + new AfterViewRefreshedEvent(PolarisEvent.createEventId(), identifier)); } } public void doCommit(ViewMetadata base, ViewMetadata metadata) { polarisEventListener.onBeforeViewCommited( - new BeforeViewCommitedEvent(PolarisEvent.createEventId(), identifier, base, metadata), - callContext, - securityContext); + new BeforeViewCommitedEvent(PolarisEvent.createEventId(), identifier, base, metadata)); // TODO: Maybe avoid writing metadata if there's definitely a transaction conflict LOGGER.debug("doCommit for view {} with base {}, metadata {}", identifier, base, metadata); @@ -1941,9 +1926,7 @@ public void doCommit(ViewMetadata base, ViewMetadata metadata) { } polarisEventListener.onAfterViewCommited( - new AfterViewCommitedEvent(PolarisEvent.createEventId(), identifier, base, metadata), - callContext, - securityContext); + new AfterViewCommitedEvent(PolarisEvent.createEventId(), identifier, base, metadata)); } protected String writeNewMetadataIfRequired(ViewMetadata metadata) { diff --git a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java index a7d076803a..30aae9806e 100644 --- a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java +++ b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java @@ -403,9 +403,7 @@ public LoadTableResponse createTableDirect(Namespace namespace, CreateTableReque catalogHandlerUtils.createTable(baseCatalog, namespace, requestWithoutReservedProperties); polarisEventListener.onAfterTableCreated( new AfterTableCreatedEvent( - PolarisEvent.createEventId(), catalogName, resp.tableMetadata(), identifier), - callContext, - securityContext); + PolarisEvent.createEventId(), catalogName, resp.tableMetadata(), identifier)); return resp; } @@ -466,9 +464,7 @@ public LoadTableResponse createTableDirectWithWriteDelegation( .build(); polarisEventListener.onAfterTableCreated( new AfterTableCreatedEvent( - PolarisEvent.createEventId(), catalogName, resp.tableMetadata(), tableIdentifier), - callContext, - securityContext); + PolarisEvent.createEventId(), catalogName, resp.tableMetadata(), tableIdentifier)); return resp; } else if (table instanceof BaseMetadataTable) { // metadata tables are loaded on the client side, return NoSuchTableException for now diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java index c217745244..f2927bf6e1 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java @@ -37,6 +37,8 @@ import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; + +import jakarta.ws.rs.core.SecurityContext; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.entity.PolarisEvent; @@ -56,11 +58,13 @@ public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersis private final ConcurrentHashMap> buffer = new ConcurrentHashMap<>(); - private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + private final ScheduledExecutorService executor; private final ConcurrentHashMap, Integer> futures = new ConcurrentHashMap<>(); private final Duration timeToFlush; private final int maxBufferSize; + @Inject CallContext callContext; + @Context SecurityContext securityContext; @Context ContainerRequestContext containerRequestContext; private record EventAndContext(PolarisEvent polarisEvent, PolarisCallContext callContext) {} @@ -75,20 +79,21 @@ public InMemoryBufferPolarisPersistenceEventListener( this.timeToFlush = eventListenerConfiguration.bufferTime().orElse(Duration.of(30, ChronoUnit.SECONDS)); this.maxBufferSize = eventListenerConfiguration.maxBufferSize().orElse(5); // 5 events default + + executor = Executors.newSingleThreadScheduledExecutor(); } @PostConstruct void start() { futures.put( executor.scheduleAtFixedRate( - this::runCleanup, 0, timeToFlush.toMillis(), TimeUnit.MILLISECONDS), - 1); + this::runCleanup, 0, timeToFlush.toMillis(), TimeUnit.MILLISECONDS), 1); } void runCleanup() { for (String realmId : buffer.keySet()) { try { - checkAndFlushBufferIfNecessary(realmId); + checkAndFlushBufferIfNecessary(realmId, false); } catch (Exception e) { LOGGER.debug("Buffer checking task failed for realm ({}): {}", realmId, e); } @@ -105,6 +110,18 @@ void runCleanup() { void shutdown() { futures.keySet().forEach(future -> future.cancel(false)); executor.shutdownNow(); + + try { + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + LOGGER.warn("Executor did not shut down cleanly"); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } finally { + for (String realmId : buffer.keySet()) { + checkAndFlushBufferIfNecessary(realmId, true); + } + } } @Override @@ -116,51 +133,49 @@ String getRequestId() { } @Override - void addToBuffer(PolarisEvent polarisEvent, CallContext callCtx) { - String realmId = callCtx.getRealmContext().getRealmIdentifier(); + void addToBuffer(PolarisEvent polarisEvent) { + String realmId = callContext.getRealmContext().getRealmIdentifier(); buffer .computeIfAbsent(realmId, k -> new ConcurrentLinkedQueue<>()) - .add(new EventAndContext(polarisEvent, callCtx.getPolarisCallContext().copy())); - futures.put(executor.submit(() -> checkAndFlushBufferIfNecessary(realmId)), 1); + .add(new EventAndContext(polarisEvent, callContext.getPolarisCallContext().copy())); + if (buffer.get(realmId).size() >= maxBufferSize) { + futures.put(executor.submit(() -> checkAndFlushBufferIfNecessary(realmId, true)), 1); + } } @VisibleForTesting - public void checkAndFlushBufferIfNecessary(String realmId) { + void checkAndFlushBufferIfNecessary(String realmId, boolean forceFlush) { ConcurrentLinkedQueue queue = buffer.get(realmId); if (queue == null || queue.isEmpty()) { return; } - // Given that we are using a ConcurrentLinkedQueue, this should not lock any calls to `add` on - // the queue. - synchronized (queue) { - // Double-check inside synchronized block - if (queue.isEmpty()) { - return; - } + EventAndContext head = queue.peek(); + if (head == null) { + return; + } + + Duration elapsed = Duration.ofMillis(clock.millis() - head.polarisEvent.getTimestampMs()); - EventAndContext head = queue.peek(); - if (head == null) { + if (elapsed.compareTo(timeToFlush) > 0 || queue.size() >= maxBufferSize || forceFlush) { + // Atomically replace old queue with new queue + boolean replaced = buffer.replace(realmId, queue, new ConcurrentLinkedQueue<>()); + if (!replaced) { + // Another thread concurrently modified the buffer, so do not continue return; } - Duration elapsed = Duration.ofMillis(clock.millis() - head.polarisEvent.getTimestampMs()); - - if (elapsed.compareTo(timeToFlush) > 0 || queue.size() >= maxBufferSize) { - // Atomically replace old queue with new queue - boolean replaced = buffer.replace(realmId, queue, new ConcurrentLinkedQueue<>()); - if (!replaced) { - // Another thread concurrently modified the buffer, so do not continue - return; - } - - metaStoreManagerFactory - .getOrCreateMetaStoreManager(() -> realmId) - .writeEvents( - head.callContext(), - new ArrayList<>(queue.stream().map(EventAndContext::polarisEvent).toList())); - } + metaStoreManagerFactory + .getOrCreateMetaStoreManager(() -> realmId) + .writeEvents( + head.callContext(), + new ArrayList<>(queue.stream().map(EventAndContext::polarisEvent).toList())); } } + + @Override + ContextSpecificInformation getContextSpecificInformation() { + return new ContextSpecificInformation(callContext.getPolarisCallContext().getClock().millis(), securityContext.getUserPrincipal() == null ? null : securityContext.getUserPrincipal().getName()); + } } diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java index 28fa5d4cf0..2d13194dcf 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java @@ -18,8 +18,6 @@ */ package org.apache.polaris.service.events.listeners; -import jakarta.ws.rs.core.SecurityContext; -import org.apache.polaris.core.context.CallContext; import org.apache.polaris.service.events.AfterCatalogCreatedEvent; import org.apache.polaris.service.events.AfterTableCommitedEvent; import org.apache.polaris.service.events.AfterTableCreatedEvent; @@ -41,52 +39,41 @@ public abstract class PolarisEventListener { /** {@link BeforeRequestRateLimitedEvent} */ - public void onBeforeRequestRateLimited( - BeforeRequestRateLimitedEvent event, SecurityContext securityContext) {} + public void onBeforeRequestRateLimited(BeforeRequestRateLimitedEvent event) {} /** {@link BeforeTableCommitedEvent} */ - public void onBeforeTableCommited( - BeforeTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onBeforeTableCommited(BeforeTableCommitedEvent event) {} /** {@link AfterTableCommitedEvent} */ - public void onAfterTableCommited( - AfterTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterTableCommited(AfterTableCommitedEvent event) {} /** {@link BeforeViewCommitedEvent} */ - public void onBeforeViewCommited( - BeforeViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onBeforeViewCommited(BeforeViewCommitedEvent event) {} /** {@link AfterViewCommitedEvent} */ - public void onAfterViewCommited( - AfterViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterViewCommited(AfterViewCommitedEvent event) {} /** {@link BeforeTableRefreshedEvent} */ - public void onBeforeTableRefreshed( - BeforeTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onBeforeTableRefreshed(BeforeTableRefreshedEvent event) {} /** {@link AfterTableRefreshedEvent} */ - public void onAfterTableRefreshed( - AfterTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterTableRefreshed(AfterTableRefreshedEvent event) {} /** {@link BeforeViewRefreshedEvent} */ - public void onBeforeViewRefreshed( - BeforeViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onBeforeViewRefreshed(BeforeViewRefreshedEvent event) {} /** {@link AfterViewRefreshedEvent} */ - public void onAfterViewRefreshed( - AfterViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterViewRefreshed(AfterViewRefreshedEvent event) {} /** {@link BeforeTaskAttemptedEvent} */ - public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event, CallContext callCtx) {} + public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event) {} /** {@link AfterTaskAttemptedEvent} */ - public void onAfterTaskAttempted(AfterTaskAttemptedEvent event, CallContext callCtx) {} + public void onAfterTaskAttempted(AfterTaskAttemptedEvent event) {} /** {@link AfterTableCreatedEvent} */ - public void onAfterTableCreated( - AfterTableCreatedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterTableCreated(AfterTableCreatedEvent event) {} /** {@link AfterCatalogCreatedEvent} */ - public void onAfterCatalogCreated( - AfterCatalogCreatedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterCatalogCreated(AfterCatalogCreatedEvent event) {} } diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java index 9b34cd07aa..4dd15f47a0 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java @@ -19,10 +19,8 @@ package org.apache.polaris.service.events.listeners; -import jakarta.ws.rs.core.SecurityContext; import java.util.Map; import org.apache.iceberg.TableMetadataParser; -import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.service.events.AfterCatalogCreatedEvent; import org.apache.polaris.service.events.AfterTableCommitedEvent; @@ -40,58 +38,49 @@ public abstract class PolarisPersistenceEventListener extends PolarisEventListener { @Override - public final void onBeforeRequestRateLimited( - BeforeRequestRateLimitedEvent event, SecurityContext securityContext) {} + public final void onBeforeRequestRateLimited(BeforeRequestRateLimitedEvent event) {} @Override - public void onBeforeTableCommited( - BeforeTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onBeforeTableCommited(BeforeTableCommitedEvent event) {} @Override - public void onAfterTableCommited( - AfterTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterTableCommited(AfterTableCommitedEvent event) {} @Override - public void onBeforeViewCommited( - BeforeViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onBeforeViewCommited(BeforeViewCommitedEvent event) {} @Override - public void onAfterViewCommited( - AfterViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterViewCommited(AfterViewCommitedEvent event) {} @Override - public void onBeforeTableRefreshed( - BeforeTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onBeforeTableRefreshed(BeforeTableRefreshedEvent event) {} @Override - public void onAfterTableRefreshed( - AfterTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterTableRefreshed(AfterTableRefreshedEvent event) {} @Override - public void onBeforeViewRefreshed( - BeforeViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onBeforeViewRefreshed(BeforeViewRefreshedEvent event) {} @Override - public void onAfterViewRefreshed( - AfterViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) {} + public void onAfterViewRefreshed(AfterViewRefreshedEvent event) {} @Override - public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event, CallContext callCtx) {} + public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event) {} @Override - public void onAfterTaskAttempted(AfterTaskAttemptedEvent event, CallContext callCtx) {} + public void onAfterTaskAttempted(AfterTaskAttemptedEvent event) {} @Override - public void onAfterTableCreated( - AfterTableCreatedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onAfterTableCreated(AfterTableCreatedEvent event) { + ContextSpecificInformation contextSpecificInformation = getContextSpecificInformation(); org.apache.polaris.core.entity.PolarisEvent polarisEvent = new org.apache.polaris.core.entity.PolarisEvent( event.catalogName(), event.eventId(), getRequestId(), event.getClass().getSimpleName(), - getTimestamp(callCtx), - getUsername(securityContext), + contextSpecificInformation.timestamp(), + contextSpecificInformation.principalName(), PolarisEvent.ResourceType.TABLE, event.identifier().toString()); Map additionalParameters = @@ -101,37 +90,30 @@ public void onAfterTableCreated( "metadata", TableMetadataParser.toJson(event.metadata())); polarisEvent.setAdditionalProperties(additionalParameters); - - addToBuffer(polarisEvent, callCtx); + addToBuffer(polarisEvent); } @Override - public void onAfterCatalogCreated( - AfterCatalogCreatedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onAfterCatalogCreated(AfterCatalogCreatedEvent event) { + ContextSpecificInformation contextSpecificInformation = getContextSpecificInformation(); org.apache.polaris.core.entity.PolarisEvent polarisEvent = - new PolarisEvent( - event.catalogName(), - event.eventId(), - getRequestId(), - event.getClass().getSimpleName(), - getTimestamp(callCtx), - getUsername(securityContext), - PolarisEvent.ResourceType.CATALOG, - event.catalogName()); - addToBuffer(polarisEvent, callCtx); + new PolarisEvent( + event.catalogName(), + event.eventId(), + getRequestId(), + event.getClass().getSimpleName(), + contextSpecificInformation.timestamp(), + contextSpecificInformation.principalName(), + PolarisEvent.ResourceType.CATALOG, + event.catalogName()); + addToBuffer(polarisEvent); } - private long getTimestamp(CallContext callCtx) { - return callCtx.getPolarisCallContext().getClock().millis(); - } + protected record ContextSpecificInformation(long timestamp, String principalName) {} - abstract String getRequestId(); + abstract ContextSpecificInformation getContextSpecificInformation(); - private String getUsername(SecurityContext securityContext) { - return securityContext.getUserPrincipal() == null - ? null - : securityContext.getUserPrincipal().getName(); - } + abstract String getRequestId(); - abstract void addToBuffer(org.apache.polaris.core.entity.PolarisEvent event, CallContext callCtx); + abstract void addToBuffer(org.apache.polaris.core.entity.PolarisEvent event); } diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java index 9ca53a8731..cf95452de3 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java @@ -21,10 +21,8 @@ import com.google.common.collect.Streams; import io.smallrye.common.annotation.Identifier; import jakarta.enterprise.context.ApplicationScoped; -import jakarta.ws.rs.core.SecurityContext; import java.util.ArrayList; import java.util.List; -import org.apache.polaris.core.context.CallContext; import org.apache.polaris.service.events.AfterTableCommitedEvent; import org.apache.polaris.service.events.AfterTableRefreshedEvent; import org.apache.polaris.service.events.AfterTaskAttemptedEvent; @@ -50,66 +48,57 @@ public T getLatest(Class type) { } @Override - public void onBeforeRequestRateLimited( - BeforeRequestRateLimitedEvent event, SecurityContext securityContext) { + public void onBeforeRequestRateLimited(BeforeRequestRateLimitedEvent event) { history.add(event); } @Override - public void onBeforeTableCommited( - BeforeTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onBeforeTableCommited(BeforeTableCommitedEvent event) { history.add(event); } @Override - public void onAfterTableCommited( - AfterTableCommitedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onAfterTableCommited(AfterTableCommitedEvent event) { history.add(event); } @Override - public void onBeforeViewCommited( - BeforeViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onBeforeViewCommited(BeforeViewCommitedEvent event) { history.add(event); } @Override - public void onAfterViewCommited( - AfterViewCommitedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onAfterViewCommited(AfterViewCommitedEvent event) { history.add(event); } @Override - public void onBeforeTableRefreshed( - BeforeTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onBeforeTableRefreshed(BeforeTableRefreshedEvent event) { history.add(event); } @Override - public void onAfterTableRefreshed( - AfterTableRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onAfterTableRefreshed(AfterTableRefreshedEvent event) { history.add(event); } @Override - public void onBeforeViewRefreshed( - BeforeViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onBeforeViewRefreshed(BeforeViewRefreshedEvent event) { history.add(event); } @Override - public void onAfterViewRefreshed( - AfterViewRefreshedEvent event, CallContext callCtx, SecurityContext securityContext) { + public void onAfterViewRefreshed(AfterViewRefreshedEvent event) { history.add(event); } @Override - public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event, CallContext callCtx) { + public void onBeforeTaskAttempted(BeforeTaskAttemptedEvent event) { history.add(event); } @Override - public void onAfterTaskAttempted(AfterTaskAttemptedEvent event, CallContext callCtx) { + public void onAfterTaskAttempted(AfterTaskAttemptedEvent event) { history.add(event); } } diff --git a/service/common/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java b/service/common/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java index d82f8da621..e1d4d42465 100644 --- a/service/common/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java +++ b/service/common/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java @@ -59,8 +59,7 @@ public void filter(ContainerRequestContext ctx) throws IOException { new BeforeRequestRateLimitedEvent( PolarisEvent.createEventId(), ctx.getMethod(), - ctx.getUriInfo().getAbsolutePath().toString()), - ctx.getSecurityContext()); + ctx.getUriInfo().getAbsolutePath().toString())); ctx.abortWith(Response.status(Response.Status.TOO_MANY_REQUESTS).build()); LOGGER.atDebug().log("Rate limiting request"); } diff --git a/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java b/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java index 87496e3eb2..5e1ac1da80 100644 --- a/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java +++ b/service/common/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java @@ -119,7 +119,7 @@ public void addTaskHandlerContext(long taskEntityId, CallContext callContext) { protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { polarisEventListener.onBeforeTaskAttempted( - new BeforeTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt), ctx); + new BeforeTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt)); boolean success = false; try { @@ -163,8 +163,7 @@ protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { } } finally { polarisEventListener.onAfterTaskAttempted( - new AfterTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt, success), - ctx); + new AfterTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt, success)); } } } diff --git a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index b76cb664dc..f67ae7f880 100644 --- a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -75,6 +75,7 @@ public void setUp() { when(eventListenerConfiguration.bufferTime()) .thenReturn(Optional.of(CONFIG_TIME_TO_FLUSH_IN_MS)); + clock = MutableClock.of( Instant.ofEpochSecond(0), ZoneOffset.UTC); // Use 0 Epoch Time to make it easier to test @@ -82,6 +83,15 @@ public void setUp() { eventListener = new InMemoryBufferPolarisPersistenceEventListener( metaStoreManagerFactory, clock, eventListenerConfiguration); + + // Use reflection to set the callContext field + try { + java.lang.reflect.Field field = InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField("callContext"); + field.setAccessible(true); + field.set(eventListener, callContext); + } catch (Exception e) { + throw new RuntimeException("Failed to set callContext field", e); + } } @Test @@ -91,7 +101,7 @@ public void testAddToBufferFlushesAfterConfiguredTime() { // Push clock forwards to flush the buffer clock.add(CONFIG_TIME_TO_FLUSH_IN_MS.multipliedBy(2)); - eventListener.checkAndFlushBufferIfNecessary(realmId); + eventListener.checkAndFlushBufferIfNecessary(realmId, false); verify(polarisMetaStoreManager, times(1)) .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); } @@ -106,12 +116,12 @@ public void testAddToBufferFlushesAfterMaxEvents() { PolarisEvent triggeringEvent = createSampleEvent(); RealmContext realmContext = () -> realm1; when(callContext.getRealmContext()).thenReturn(realmContext); - eventListener.addToBuffer(triggeringEvent, callContext); + eventListener.addToBuffer(triggeringEvent); eventsAddedToBuffer.add(triggeringEvent); // Calling checkAndFlushBufferIfNecessary manually to replicate the behavior of the executor // service - eventListener.checkAndFlushBufferIfNecessary(realm1); + eventListener.checkAndFlushBufferIfNecessary(realm1, false); verify(polarisMetaStoreManager, times(1)) .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); verify(polarisMetaStoreManager, times(0)) @@ -137,7 +147,7 @@ public void testCheckAndFlushBufferIfNecessaryIsThreadSafe() throws Exception { new Thread( () -> { try { - eventListener.checkAndFlushBufferIfNecessary(realmId); + eventListener.checkAndFlushBufferIfNecessary(realmId, false); } catch (Exception e) { exceptions.add(e); } @@ -253,7 +263,7 @@ private List addEventsWithoutTriggeringFlush(String realmId) { RealmContext realmContext = () -> realmId; when(callContext.getRealmContext()).thenReturn(realmContext); for (PolarisEvent realmEvent : realmEvents) { - eventListener.addToBuffer(realmEvent, callContext); + eventListener.addToBuffer(realmEvent); } verify(polarisMetaStoreManager, times(0)).writeEvents(Mockito.any(), Mockito.any()); return realmEvents; From 69acffd31c6093d3ab323870eadb50f73ea8f15f Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Thu, 7 Aug 2025 16:16:36 -0700 Subject: [PATCH 08/17] manually tested and verified --- ...arisInMemoryBufferEventListenerConfiguration.java | 12 +++++++----- ...MemoryBufferPersistenceListenerConfiguration.java | 4 ++-- ...nMemoryBufferPolarisPersistenceEventListener.java | 5 ++--- ...oryBufferPolarisPersistenceEventListenerTest.java | 6 ++---- 4 files changed, 13 insertions(+), 14 deletions(-) diff --git a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java index 6ddec631eb..732ae6cf09 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/quarkus/events/QuarkusPolarisInMemoryBufferEventListenerConfiguration.java @@ -19,26 +19,27 @@ package org.apache.polaris.service.quarkus.events; +import io.quarkus.runtime.annotations.StaticInitSafe; import io.smallrye.config.ConfigMapping; import io.smallrye.config.WithDefault; import io.smallrye.config.WithName; import jakarta.enterprise.context.ApplicationScoped; import java.time.Duration; -import java.util.Optional; import org.apache.polaris.service.events.listeners.InMemoryBufferPersistenceListenerConfiguration; -@ConfigMapping(prefix = "polaris.event-listener.persistence-in-memory-buffer") +@StaticInitSafe @ApplicationScoped +@ConfigMapping(prefix = "polaris.event-listener.persistence-in-memory-buffer") public interface QuarkusPolarisInMemoryBufferEventListenerConfiguration - extends InMemoryBufferPersistenceListenerConfiguration { + extends InMemoryBufferPersistenceListenerConfiguration { /** * @return the buffer time in milliseconds */ @Override @WithName("buffer-time") @WithDefault("5000ms") - Optional bufferTime(); + Duration bufferTime(); /** * @return the maximum number of cached entries @@ -46,5 +47,6 @@ public interface QuarkusPolarisInMemoryBufferEventListenerConfiguration @Override @WithName("max-buffer-size") @WithDefault("5") - Optional maxBufferSize(); + int maxBufferSize(); } + diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPersistenceListenerConfiguration.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPersistenceListenerConfiguration.java index 34ea164e18..a73c25e4cb 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPersistenceListenerConfiguration.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPersistenceListenerConfiguration.java @@ -33,10 +33,10 @@ public interface InMemoryBufferPersistenceListenerConfiguration { /** * @return the buffer time in milliseconds */ - Optional bufferTime(); + Duration bufferTime(); /** * @return the maximum number of cached entries */ - Optional maxBufferSize(); + int maxBufferSize(); } diff --git a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java index f2927bf6e1..290772e8dd 100644 --- a/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java +++ b/service/common/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java @@ -76,9 +76,8 @@ public InMemoryBufferPolarisPersistenceEventListener( InMemoryBufferPersistenceListenerConfiguration eventListenerConfiguration) { this.metaStoreManagerFactory = metaStoreManagerFactory; this.clock = clock; - this.timeToFlush = - eventListenerConfiguration.bufferTime().orElse(Duration.of(30, ChronoUnit.SECONDS)); - this.maxBufferSize = eventListenerConfiguration.maxBufferSize().orElse(5); // 5 events default + this.timeToFlush = eventListenerConfiguration.bufferTime(); + this.maxBufferSize = eventListenerConfiguration.maxBufferSize(); executor = Executors.newSingleThreadScheduledExecutor(); } diff --git a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index f67ae7f880..707ace6866 100644 --- a/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/service/common/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -70,10 +70,8 @@ public void setUp() { InMemoryBufferPersistenceListenerConfiguration eventListenerConfiguration = Mockito.mock(InMemoryBufferPersistenceListenerConfiguration.class); - when(eventListenerConfiguration.maxBufferSize()) - .thenReturn(Optional.of(CONFIG_MAX_BUFFER_SIZE)); - when(eventListenerConfiguration.bufferTime()) - .thenReturn(Optional.of(CONFIG_TIME_TO_FLUSH_IN_MS)); + when(eventListenerConfiguration.maxBufferSize()).thenReturn(CONFIG_MAX_BUFFER_SIZE); + when(eventListenerConfiguration.bufferTime()).thenReturn(CONFIG_TIME_TO_FLUSH_IN_MS); clock = From b1438c36dbea30175062ba9a319bc3adae359266 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Mon, 11 Aug 2025 19:51:01 -0700 Subject: [PATCH 09/17] address comment from @eric-maynard --- .../events/listeners/PolarisPersistenceEventListener.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java index e05a860f8d..9ae681e0cc 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java @@ -37,6 +37,8 @@ import org.apache.polaris.service.events.BeforeViewRefreshedEvent; public abstract class PolarisPersistenceEventListener extends PolarisEventListener { + + // TODO: Ensure all events (except RateLimiter ones call `addToBuffer`) @Override public final void onBeforeRequestRateLimited(BeforeRequestRateLimitedEvent event) {} From b1e43ccd8cb1d59ce660383cd8fafa9ba9d433b6 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Mon, 11 Aug 2025 19:59:39 -0700 Subject: [PATCH 10/17] typo --- .../events/listeners/PolarisPersistenceEventListener.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java index 9ae681e0cc..fe7f6ebb83 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java @@ -38,7 +38,7 @@ public abstract class PolarisPersistenceEventListener extends PolarisEventListener { - // TODO: Ensure all events (except RateLimiter ones call `addToBuffer`) + // TODO: Ensure all events (except RateLimiter ones) call `addToBuffer` @Override public final void onBeforeRequestRateLimited(BeforeRequestRateLimitedEvent event) {} From a8657a58f32e42389f2803d283c20c0511613ad0 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Fri, 29 Aug 2025 00:01:24 -0700 Subject: [PATCH 11/17] Revision based on review from @adutra --- .../jdbc/JdbcBasePersistenceImpl.java | 2 +- .../polaris/core/entity/PolarisEvent.java | 1 + .../AtomicOperationMetaStoreManager.java | 10 -- .../service/admin/PolarisServiceImpl.java | 3 +- .../catalog/iceberg/IcebergCatalog.java | 28 ++--- .../iceberg/IcebergCatalogHandler.java | 8 +- .../events/AfterCatalogCreatedEvent.java | 2 +- .../events/AfterTableCommitedEvent.java | 9 +- .../events/AfterTableCreatedEvent.java | 2 +- .../events/AfterTableRefreshedEvent.java | 4 +- .../events/AfterTaskAttemptedEvent.java | 3 +- .../events/AfterViewCommitedEvent.java | 10 +- .../events/AfterViewRefreshedEvent.java | 3 +- .../events/BeforeRequestRateLimitedEvent.java | 2 +- .../events/BeforeTableCommitedEvent.java | 7 +- .../events/BeforeTableRefreshedEvent.java | 3 +- .../events/BeforeTaskAttemptedEvent.java | 2 +- .../events/BeforeViewCommitedEvent.java | 10 +- .../events/BeforeViewRefreshedEvent.java | 3 +- ...currentLinkedQueueWithApproximateSize.java | 50 ++++++++ ...BufferPolarisPersistenceEventListener.java | 60 ++++----- .../PolarisPersistenceEventListener.java | 16 +-- .../ratelimiter/RateLimiterFilter.java | 5 +- .../service/task/TaskExecutorImpl.java | 7 +- .../catalog/AbstractIcebergCatalogTest.java | 4 +- ...erPolarisPersistenceEventListenerTest.java | 116 +++++++++++------- 26 files changed, 214 insertions(+), 156 deletions(-) create mode 100644 runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java index 0cae142938..156da56d8c 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java @@ -258,7 +258,7 @@ public void writeEvents(@Nonnull List events) { QueryGenerator.generateInsertQuery( ModelEvent.ALL_COLUMNS, ModelEvent.TABLE_NAME, - ModelEvent.fromEvent(events.get(0)) + ModelEvent.fromEvent(events.getFirst()) .toMap(datasourceOperations.getDatabaseType()) .values() .stream() diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java index ca80df7795..010108193a 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEvent.java @@ -25,6 +25,7 @@ import java.util.Map; public class PolarisEvent { + // TODO: Look into using the CDI-managed `ObjectMapper` object public static final String EMPTY_MAP_STRING = "{}"; // to serialize/deserialize properties diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java index e127ef6804..77c29c56b0 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java @@ -1960,14 +1960,4 @@ private List loadPoliciesFromMappingRecords( .collect(Collectors.toList()); return ms.lookupEntities(callCtx, policyEntityIds); } - - @Nonnull - @Override - public void writeEvents( - @Nonnull PolarisCallContext callCtx, @Nonnull List polarisEvents) { - // get metastore we should be using - BasePersistence ms = callCtx.getMetaStore(); - - ms.writeEvents(polarisEvents); - } } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java b/runtime/service/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java index 59293d7e10..da468734e2 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java @@ -85,7 +85,6 @@ import org.apache.polaris.service.admin.api.PolarisPrincipalsApiService; import org.apache.polaris.service.config.ReservedProperties; import org.apache.polaris.service.events.AfterCatalogCreatedEvent; -import org.apache.polaris.service.events.PolarisEvent; import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.types.PolicyIdentifier; import org.slf4j.Logger; @@ -174,7 +173,7 @@ public Response createCatalog( Catalog newCatalog = CatalogEntity.of(adminService.createCatalog(request)).asCatalog(); LOGGER.info("Created new catalog {}", newCatalog); polarisEventListener.onAfterCatalogCreated( - new AfterCatalogCreatedEvent(PolarisEvent.createEventId(), newCatalog.getName())); + new AfterCatalogCreatedEvent(newCatalog.getName())); return Response.status(Response.Status.CREATED).build(); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java index 1904f85781..a8fc9c1509 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java @@ -1384,7 +1384,7 @@ public void doRefresh() { disableRefresh(); } else { polarisEventListener.onBeforeTableRefreshed( - new BeforeTableRefreshedEvent(PolarisEvent.createEventId(), tableIdentifier)); + new BeforeTableRefreshedEvent(catalogName, tableIdentifier)); refreshFromMetadataLocation( latestLocation, SHOULD_RETRY_REFRESH_PREDICATE, @@ -1405,17 +1405,15 @@ public void doRefresh() { return TableMetadataParser.read(fileIO, metadataLocation); }); polarisEventListener.onAfterTableRefreshed( - new AfterTableRefreshedEvent(PolarisEvent.createEventId(), tableIdentifier)); + new AfterTableRefreshedEvent(catalogName, tableIdentifier)); } } public void doCommit(TableMetadata base, TableMetadata metadata) { - polarisEventListener.onBeforeTableCommited( - new BeforeTableCommitedEvent( - PolarisEvent.createEventId(), tableIdentifier, base, metadata)); + polarisEventListener.onBeforeTableCommited(new BeforeTableCommitedEvent(tableIdentifier, base, metadata)); LOGGER.debug( - "doCommit for table {} with base {}, metadata {}", tableIdentifier, base, metadata); + "doCommit for table {} with metadataBefore {}, metadataAfter {}", tableIdentifier, base, metadata); // TODO: Maybe avoid writing metadata if there's definitely a transaction conflict if (null == base && !namespaceExists(tableIdentifier.namespace())) { throw new NoSuchNamespaceException( @@ -1553,9 +1551,7 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { updateTableLike(tableIdentifier, entity); } - polarisEventListener.onAfterTableCommited( - new AfterTableCommitedEvent( - PolarisEvent.createEventId(), tableIdentifier, base, metadata)); + polarisEventListener.onAfterTableCommited(new AfterTableCommitedEvent(catalogName, tableIdentifier, base, metadata)); } @Override @@ -1746,8 +1742,7 @@ public void doRefresh() { if (latestLocation == null) { disableRefresh(); } else { - polarisEventListener.onBeforeViewRefreshed( - new BeforeViewRefreshedEvent(PolarisEvent.createEventId(), identifier)); + polarisEventListener.onBeforeViewRefreshed(new BeforeViewRefreshedEvent(catalogName, identifier)); refreshFromMetadataLocation( latestLocation, SHOULD_RETRY_REFRESH_PREDICATE, @@ -1769,17 +1764,15 @@ public void doRefresh() { return ViewMetadataParser.read(fileIO.newInputFile(metadataLocation)); }); - polarisEventListener.onAfterViewRefreshed( - new AfterViewRefreshedEvent(PolarisEvent.createEventId(), identifier)); + polarisEventListener.onAfterViewRefreshed(new AfterViewRefreshedEvent(catalogName, identifier)); } } public void doCommit(ViewMetadata base, ViewMetadata metadata) { - polarisEventListener.onBeforeViewCommited( - new BeforeViewCommitedEvent(PolarisEvent.createEventId(), identifier, base, metadata)); + polarisEventListener.onBeforeViewCommited(new BeforeViewCommitedEvent(catalogName, identifier, base, metadata)); // TODO: Maybe avoid writing metadata if there's definitely a transaction conflict - LOGGER.debug("doCommit for view {} with base {}, metadata {}", identifier, base, metadata); + LOGGER.debug("doCommit for view {} with metadataBefore {}, metadataAfter {}", identifier, base, metadata); if (null == base && !namespaceExists(identifier.namespace())) { throw new NoSuchNamespaceException( "Cannot create view '%s'. Namespace does not exist: '%s'", @@ -1872,8 +1865,7 @@ public void doCommit(ViewMetadata base, ViewMetadata metadata) { updateTableLike(identifier, entity); } - polarisEventListener.onAfterViewCommited( - new AfterViewCommitedEvent(PolarisEvent.createEventId(), identifier, base, metadata)); + polarisEventListener.onAfterViewCommited(new AfterViewCommitedEvent(catalogName, identifier, base, metadata)); } protected String writeNewMetadataIfRequired(ViewMetadata metadata) { diff --git a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java index 2e8fe7ad20..2659cf2114 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java @@ -389,9 +389,7 @@ public LoadTableResponse createTableDirect(Namespace namespace, CreateTableReque .build(); LoadTableResponse resp = catalogHandlerUtils.createTable(baseCatalog, namespace, requestWithoutReservedProperties); - polarisEventListener.onAfterTableCreated( - new AfterTableCreatedEvent( - PolarisEvent.createEventId(), catalogName, resp.tableMetadata(), identifier)); + polarisEventListener.onAfterTableCreated(new AfterTableCreatedEvent(catalogName, identifier, resp.tableMetadata())); return resp; } @@ -445,9 +443,7 @@ public LoadTableResponse createTableDirectWithWriteDelegation( PolarisStorageActions.LIST), SNAPSHOTS_ALL) .build(); - polarisEventListener.onAfterTableCreated( - new AfterTableCreatedEvent( - PolarisEvent.createEventId(), catalogName, resp.tableMetadata(), tableIdentifier)); + polarisEventListener.onAfterTableCreated(new AfterTableCreatedEvent(catalogName, tableIdentifier, resp.tableMetadata())); return resp; } else if (table instanceof BaseMetadataTable) { // metadata tables are loaded on the client side, return NoSuchTableException for now diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java index dd125048a0..740a3d5ea3 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java @@ -20,5 +20,5 @@ package org.apache.polaris.service.events; /** Emitted Polaris creates a catalog. */ -public record AfterCatalogCreatedEvent(String eventId, String catalogName) +public record AfterCatalogCreatedEvent(String catalogName) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java index c1988d79d7..ee744659cb 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java @@ -25,10 +25,11 @@ * Emitted after Polaris performs a commit to a table. This is not emitted if there's an exception * while committing. * - * @param identifier The identifier. - * @param base The old metadata. - * @param metadata The new metadata. + * @param catalogName The catalog name where this table exists. + * @param identifier The identifier. + * @param base The old metadata. + * @param metadata The new metadata. */ public record AfterTableCommitedEvent( - String eventId, TableIdentifier identifier, TableMetadata base, TableMetadata metadata) + String catalogName, TableIdentifier identifier, TableMetadata base, TableMetadata metadata) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java index f55ad5aac3..7bfb5f3b4e 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java @@ -24,5 +24,5 @@ /** Emitted when Polaris creates a table. */ public record AfterTableCreatedEvent( - String eventId, String catalogName, TableMetadata metadata, TableIdentifier identifier) + String catalogName, TableIdentifier identifier, TableMetadata metadata) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java index 00567b4a86..d79ab9ed6d 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java @@ -23,8 +23,8 @@ /** * Emitted after Polaris refreshes its known version of a table's metadata by fetching the latest. * - * @param eventId The event unique identifier + * @param catalogName The name of the catalog where the table is located * @param tableIdentifier The identifier of the table that was refreshed. */ -public record AfterTableRefreshedEvent(String eventId, TableIdentifier tableIdentifier) +public record AfterTableRefreshedEvent(String catalogName, TableIdentifier tableIdentifier) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java index a3e132e0cd..2068504434 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java @@ -26,5 +26,4 @@ * initial (non-retried) attempt starts counting from 1. * @param success Whether the attempt succeeded. */ -public record AfterTaskAttemptedEvent( - String eventId, long taskEntityId, int attempt, boolean success) implements PolarisEvent {} +public record AfterTaskAttemptedEvent(long taskEntityId, int attempt, boolean success) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java index 072a406a32..86ff8192b4 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java @@ -25,10 +25,10 @@ * Emitted after Polaris performs a commit to a view. This is not emitted if there's an exception * while committing. * - * @param identifier The identifier. - * @param base The old metadata. - * @param metadata The new metadata. + * @param catalogName The catalog name where the view is located + * @param identifier The identifier. + * @param base The old metadata. + * @param metadata The new metadata. */ -public record AfterViewCommitedEvent( - String eventId, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) +public record AfterViewCommitedEvent(String catalogName, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java index edb13d2b20..b44fa2a510 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewRefreshedEvent.java @@ -23,7 +23,8 @@ /** * Emitted after Polaris refreshes its known version of a view's metadata by fetching the latest. * + * @param catalogName * @param viewIdentifier The identifier of the view that was refreshed. */ -public record AfterViewRefreshedEvent(String eventId, TableIdentifier viewIdentifier) +public record AfterViewRefreshedEvent(String catalogName, TableIdentifier viewIdentifier) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeRequestRateLimitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeRequestRateLimitedEvent.java index d3df44f3f3..1d9780ebe7 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeRequestRateLimitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeRequestRateLimitedEvent.java @@ -24,5 +24,5 @@ * @param method The request's HTTP method * @param absolutePath The request's absolute path */ -public record BeforeRequestRateLimitedEvent(String eventId, String method, String absolutePath) +public record BeforeRequestRateLimitedEvent(String method, String absolutePath) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java index ed1981e3b0..0e98c06337 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java @@ -27,9 +27,8 @@ * fail Polaris-side validation checks. * * @param tableIdentifier The identifier. - * @param base The old metadata. - * @param metadata The new metadata. + * @param metadataBefore The old metadata. + * @param metadataAfter The new metadata. */ -public record BeforeTableCommitedEvent( - String eventId, TableIdentifier tableIdentifier, TableMetadata base, TableMetadata metadata) +public record BeforeTableCommitedEvent(TableIdentifier tableIdentifier, TableMetadata metadataBefore, TableMetadata metadataAfter) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java index b50d1abb6b..0ed57e59ca 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java @@ -24,7 +24,8 @@ * Emitted when Polaris intends to refresh its known version of a table's metadata by fetching the * latest. * + * @param catalogName The name of the catalog where the view is located. * @param tableIdentifier The identifier of the table being refreshed. */ -public record BeforeTableRefreshedEvent(String eventId, TableIdentifier tableIdentifier) +public record BeforeTableRefreshedEvent(String catalogName, TableIdentifier tableIdentifier) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java index 13ecb0ce2e..7580ff81cd 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java @@ -25,5 +25,5 @@ * @param attempt The attempt number. Each retry of the task will have its own attempt number. The * initial (non-retried) attempt starts counting from 1. */ -public record BeforeTaskAttemptedEvent(String eventId, long taskEntityId, int attempt) +public record BeforeTaskAttemptedEvent(long taskEntityId, int attempt) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java index c163cc5716..a86a07fc68 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java @@ -26,10 +26,10 @@ * this event relative to the validation checks we've performed, which means the commit may still * fail Polaris-side validation checks. * - * @param identifier The identifier. - * @param base The old metadata. - * @param metadata The new metadata. + * @param catalogName The name of the catalog where the view is located. + * @param identifier The identifier. + * @param base The old metadata. + * @param metadata The new metadata. */ -public record BeforeViewCommitedEvent( - String eventId, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) +public record BeforeViewCommitedEvent(String catalogName, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java index 1b652690a9..559e5754a4 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java @@ -24,7 +24,8 @@ * Emitted when Polaris intends to refresh its known version of a view's metadata by fetching the * latest. * + * @param catalogName The name of the catalog where the view is located. * @param viewIdentifier The identifier of the view being refreshed. */ -public record BeforeViewRefreshedEvent(String eventId, TableIdentifier viewIdentifier) +public record BeforeViewRefreshedEvent(String catalogName, TableIdentifier viewIdentifier) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java new file mode 100644 index 0000000000..d24bcb0e55 --- /dev/null +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java @@ -0,0 +1,50 @@ +/* + * 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.polaris.service.events.listeners; + +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; + +public class ConcurrentLinkedQueueWithApproximateSize { + private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private final AtomicInteger size = new AtomicInteger(); + + public void add(T event) { + queue.add(event); + size.getAndIncrement(); + } + + public boolean isEmpty() { + return queue.isEmpty(); + } + + public T peek() { + return queue.peek(); + } + + public int size() { + return size.get(); + } + + public Stream stream() { + return queue.stream(); + } +} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java index 5bdc7fd093..fb1cb27801 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java @@ -29,18 +29,22 @@ import jakarta.ws.rs.core.SecurityContext; import java.time.Clock; import java.time.Duration; -import java.util.ArrayList; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; + import org.apache.polaris.core.PolarisCallContext; +import org.apache.polaris.core.PolarisDiagnostics; import org.apache.polaris.core.context.CallContext; +import org.apache.polaris.core.context.RealmContext; import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.core.persistence.BasePersistence; import org.apache.polaris.core.persistence.MetaStoreManagerFactory; +import org.apache.polaris.core.persistence.PolarisMetaStoreManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,10 +57,10 @@ public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersis private static final String REQUEST_ID_KEY = "requestId"; private final MetaStoreManagerFactory metaStoreManagerFactory; - private final ConcurrentHashMap> buffer = + private final ConcurrentHashMap> buffer = new ConcurrentHashMap<>(); private final ScheduledExecutorService executor; - private final ConcurrentHashMap, Integer> futures = new ConcurrentHashMap<>(); + private final Set> futures = ConcurrentHashMap.newKeySet(); private final Duration timeToFlush; private final int maxBufferSize; @@ -64,8 +68,7 @@ public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersis @Inject Clock clock; @Context SecurityContext securityContext; @Context ContainerRequestContext containerRequestContext; - - private record EventAndContext(PolarisEvent polarisEvent, PolarisCallContext callContext) {} + @Inject PolarisDiagnostics polarisDiagnostics; @Inject public InMemoryBufferPolarisPersistenceEventListener( @@ -82,10 +85,9 @@ public InMemoryBufferPolarisPersistenceEventListener( @PostConstruct void start() { - futures.put( + futures.add( executor.scheduleAtFixedRate( - this::runCleanup, 0, timeToFlush.toMillis(), TimeUnit.MILLISECONDS), - 1); + this::runCleanup, 0, timeToFlush.toMillis(), TimeUnit.MILLISECONDS)); } void runCleanup() { @@ -98,7 +100,7 @@ void runCleanup() { } // Clean up futures try { - futures.keySet().removeIf(future -> future.isCancelled() || future.isDone()); + futures.removeIf(Future::isDone); } catch (Exception e) { LOGGER.debug("Futures reaper task failed."); } @@ -106,14 +108,18 @@ void runCleanup() { @PreDestroy void shutdown() { - futures.keySet().forEach(future -> future.cancel(false)); - executor.shutdownNow(); + futures.forEach(future -> future.cancel(false)); + executor.shutdown(); try { if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { - LOGGER.warn("Executor did not shut down cleanly"); + executor.shutdownNow(); + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + LOGGER.warn("Executor did not shut down cleanly"); + } } } catch (InterruptedException e) { + executor.shutdownNow(); Thread.currentThread().interrupt(); } finally { for (String realmId : buffer.keySet()) { @@ -135,44 +141,42 @@ String getRequestId() { } @Override - void addToBuffer(PolarisEvent polarisEvent) { + void processEvent(PolarisEvent polarisEvent) { String realmId = callContext.getRealmContext().getRealmIdentifier(); - buffer - .computeIfAbsent(realmId, k -> new ConcurrentLinkedQueue<>()) - .add(new EventAndContext(polarisEvent, callContext.getPolarisCallContext().copy())); - if (buffer.get(realmId).size() >= maxBufferSize) { - futures.put(executor.submit(() -> checkAndFlushBufferIfNecessary(realmId, true)), 1); + ConcurrentLinkedQueueWithApproximateSize realmQueue = buffer.computeIfAbsent(realmId, k -> new ConcurrentLinkedQueueWithApproximateSize<>()); + realmQueue.add(polarisEvent); + if (realmQueue.size() >= maxBufferSize) { + futures.add(executor.submit(() -> checkAndFlushBufferIfNecessary(realmId, true))); } } @VisibleForTesting void checkAndFlushBufferIfNecessary(String realmId, boolean forceFlush) { - ConcurrentLinkedQueue queue = buffer.get(realmId); + ConcurrentLinkedQueueWithApproximateSize queue = buffer.get(realmId); if (queue == null || queue.isEmpty()) { return; } - EventAndContext head = queue.peek(); + PolarisEvent head = queue.peek(); if (head == null) { return; } - Duration elapsed = Duration.ofMillis(clock.millis() - head.polarisEvent.getTimestampMs()); + Duration elapsed = Duration.ofMillis(clock.millis() - head.getTimestampMs()); if (elapsed.compareTo(timeToFlush) > 0 || queue.size() >= maxBufferSize || forceFlush) { // Atomically replace old queue with new queue - boolean replaced = buffer.replace(realmId, queue, new ConcurrentLinkedQueue<>()); + boolean replaced = buffer.replace(realmId, queue, new ConcurrentLinkedQueueWithApproximateSize<>()); if (!replaced) { // Another thread concurrently modified the buffer, so do not continue return; } - metaStoreManagerFactory - .getOrCreateMetaStoreManager(() -> realmId) - .writeEvents( - head.callContext(), - new ArrayList<>(queue.stream().map(EventAndContext::polarisEvent).toList())); + RealmContext realmContext = () -> realmId; + PolarisMetaStoreManager metaStoreManager = metaStoreManagerFactory.getOrCreateMetaStoreManager(realmContext); + BasePersistence basePersistence = metaStoreManagerFactory.getOrCreateSession(realmContext); + metaStoreManager.writeEvents(new PolarisCallContext(realmContext, basePersistence, polarisDiagnostics), queue.stream().toList()); } } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java index fe7f6ebb83..d31beb45e6 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java @@ -75,10 +75,10 @@ public void onAfterTaskAttempted(AfterTaskAttemptedEvent event) {} @Override public void onAfterTableCreated(AfterTableCreatedEvent event) { ContextSpecificInformation contextSpecificInformation = getContextSpecificInformation(); - org.apache.polaris.core.entity.PolarisEvent polarisEvent = - new org.apache.polaris.core.entity.PolarisEvent( + PolarisEvent polarisEvent = + new PolarisEvent( event.catalogName(), - event.eventId(), + org.apache.polaris.service.events.PolarisEvent.createEventId(), getRequestId(), event.getClass().getSimpleName(), contextSpecificInformation.timestamp(), @@ -92,23 +92,23 @@ public void onAfterTableCreated(AfterTableCreatedEvent event) { "metadata", TableMetadataParser.toJson(event.metadata())); polarisEvent.setAdditionalProperties(additionalParameters); - addToBuffer(polarisEvent); + processEvent(polarisEvent); } @Override public void onAfterCatalogCreated(AfterCatalogCreatedEvent event) { ContextSpecificInformation contextSpecificInformation = getContextSpecificInformation(); - org.apache.polaris.core.entity.PolarisEvent polarisEvent = + PolarisEvent polarisEvent = new PolarisEvent( event.catalogName(), - event.eventId(), + org.apache.polaris.service.events.PolarisEvent.createEventId(), getRequestId(), event.getClass().getSimpleName(), contextSpecificInformation.timestamp(), contextSpecificInformation.principalName(), PolarisEvent.ResourceType.CATALOG, event.catalogName()); - addToBuffer(polarisEvent); + processEvent(polarisEvent); } protected record ContextSpecificInformation(long timestamp, String principalName) {} @@ -117,5 +117,5 @@ protected record ContextSpecificInformation(long timestamp, String principalName abstract String getRequestId(); - abstract void addToBuffer(org.apache.polaris.core.entity.PolarisEvent event); + abstract void processEvent(PolarisEvent event); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java b/runtime/service/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java index 920867a1c3..c1c825b587 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java @@ -56,10 +56,7 @@ public RateLimiterFilter(RateLimiter rateLimiter, PolarisEventListener polarisEv public void filter(ContainerRequestContext ctx) throws IOException { if (!rateLimiter.canProceed()) { polarisEventListener.onBeforeRequestRateLimited( - new BeforeRequestRateLimitedEvent( - PolarisEvent.createEventId(), - ctx.getMethod(), - ctx.getUriInfo().getAbsolutePath().toString())); + new BeforeRequestRateLimitedEvent(ctx.getMethod(), ctx.getUriInfo().getAbsolutePath().toString())); ctx.abortWith(Response.status(Response.Status.TOO_MANY_REQUESTS).build()); LOGGER.atDebug().log("Rate limiting request"); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java b/runtime/service/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java index 7859fed802..25d0584d7f 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java @@ -144,8 +144,7 @@ public void addTaskHandlerContext(long taskEntityId, CallContext callContext) { } protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { - polarisEventListener.onBeforeTaskAttempted( - new BeforeTaskAttemptedEvent(PolarisEvent.createEventId(), taskEntityId, attempt)); + polarisEventListener.onBeforeTaskAttempted(new BeforeTaskAttemptedEvent(taskEntityId, attempt)); boolean success = false; try { @@ -188,9 +187,7 @@ protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { .log("Unable to execute async task"); } } finally { - polarisEventListener.onAfterTaskAttempted( - new AfterTaskAttemptedEvent( - PolarisEvent.createEventId(), taskEntityId, attempt, success)); + polarisEventListener.onAfterTaskAttempted(new AfterTaskAttemptedEvent(taskEntityId, attempt, success)); } } diff --git a/runtime/service/src/test/java/org/apache/polaris/service/catalog/AbstractIcebergCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/catalog/AbstractIcebergCatalogTest.java index 13b025cbfd..e9f5075b1e 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/catalog/AbstractIcebergCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/catalog/AbstractIcebergCatalogTest.java @@ -2230,8 +2230,8 @@ public void testEventsAreEmitted() { var beforeTableEvent = testPolarisEventListener.getLatest(BeforeTableCommitedEvent.class); Assertions.assertThat(beforeTableEvent.tableIdentifier()).isEqualTo(TestData.TABLE); - Assertions.assertThat(beforeTableEvent.base().properties().get(key)).isEqualTo(valOld); - Assertions.assertThat(beforeTableEvent.metadata().properties().get(key)).isEqualTo(valNew); + Assertions.assertThat(beforeTableEvent.metadataBefore().properties().get(key)).isEqualTo(valOld); + Assertions.assertThat(beforeTableEvent.metadataAfter().properties().get(key)).isEqualTo(valNew); var afterTableEvent = testPolarisEventListener.getLatest(AfterTableCommitedEvent.class); Assertions.assertThat(afterTableEvent.identifier()).isEqualTo(TestData.TABLE); diff --git a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index f84aec8e95..d6ff90f895 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -20,7 +20,10 @@ package org.apache.polaris.service.events.listeners; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.refEq; +import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -35,14 +38,18 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.stream.Collectors; + import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.context.RealmContext; import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.persistence.MetaStoreManagerFactory; import org.apache.polaris.core.persistence.PolarisMetaStoreManager; +import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.threeten.extra.MutableClock; @@ -60,11 +67,10 @@ public void setUp() { callContext = Mockito.mock(CallContext.class); PolarisCallContext polarisCallContext = Mockito.mock(PolarisCallContext.class); when(callContext.getPolarisCallContext()).thenReturn(polarisCallContext); - when(polarisCallContext.copy()).thenReturn(polarisCallContext); MetaStoreManagerFactory metaStoreManagerFactory = Mockito.mock(MetaStoreManagerFactory.class); polarisMetaStoreManager = Mockito.mock(PolarisMetaStoreManager.class); - when(metaStoreManagerFactory.getOrCreateMetaStoreManager(Mockito.any())) + when(metaStoreManagerFactory.getOrCreateMetaStoreManager(any())) .thenReturn(polarisMetaStoreManager); InMemoryBufferEventListenerConfiguration eventListenerConfiguration = @@ -80,19 +86,11 @@ public void setUp() { new InMemoryBufferPolarisPersistenceEventListener( metaStoreManagerFactory, clock, eventListenerConfiguration); - // Use reflection to set the callContext field - try { - java.lang.reflect.Field field = - InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField("callContext"); - field.setAccessible(true); - field.set(eventListener, callContext); - } catch (Exception e) { - throw new RuntimeException("Failed to set callContext field", e); - } + eventListener.callContext = callContext; } @Test - public void testAddToBufferFlushesAfterConfiguredTime() { + public void testProcessEventFlushesAfterConfiguredTime() { String realmId = "realm1"; List eventsAddedToBuffer = addEventsWithoutTriggeringFlush(realmId); @@ -100,11 +98,11 @@ public void testAddToBufferFlushesAfterConfiguredTime() { clock.add(CONFIG_TIME_TO_FLUSH_IN_MS.multipliedBy(2)); eventListener.checkAndFlushBufferIfNecessary(realmId, false); verify(polarisMetaStoreManager, times(1)) - .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); + .writeEvents(any(), eq(eventsAddedToBuffer)); } @Test - public void testAddToBufferFlushesAfterMaxEvents() { + public void testProcessEventFlushesAfterMaxEvents() { String realm1 = "realm1"; List eventsAddedToBuffer = addEventsWithoutTriggeringFlush(realm1); List eventsAddedToBufferRealm2 = addEventsWithoutTriggeringFlush("realm2"); @@ -113,16 +111,14 @@ public void testAddToBufferFlushesAfterMaxEvents() { PolarisEvent triggeringEvent = createSampleEvent(); RealmContext realmContext = () -> realm1; when(callContext.getRealmContext()).thenReturn(realmContext); - eventListener.addToBuffer(triggeringEvent); + eventListener.processEvent(triggeringEvent); eventsAddedToBuffer.add(triggeringEvent); // Calling checkAndFlushBufferIfNecessary manually to replicate the behavior of the executor // service eventListener.checkAndFlushBufferIfNecessary(realm1, false); - verify(polarisMetaStoreManager, times(1)) - .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBuffer)); - verify(polarisMetaStoreManager, times(0)) - .writeEvents(eq(callContext.getPolarisCallContext()), eq(eventsAddedToBufferRealm2)); + verify(polarisMetaStoreManager, times(1)).writeEvents(any(), eq(eventsAddedToBuffer)); + verify(polarisMetaStoreManager, times(0)).writeEvents(any(), eq(eventsAddedToBufferRealm2)); } @Test @@ -163,8 +159,60 @@ public void testCheckAndFlushBufferIfNecessaryIsThreadSafe() throws Exception { "Exceptions occurred in concurrent checkAndFlushBufferIfNecessary: ", exceptions.peek()); } // Only one flush should occur - verify(polarisMetaStoreManager, times(1)) - .writeEvents(eq(callContext.getPolarisCallContext()), eq(events)); + verify(polarisMetaStoreManager, times(1)).writeEvents(any(), eq(events)); + } + + @Test + public void testProcessEventIsThreadSafe() throws Exception { + String realmId = "realm1"; + when(callContext.getRealmContext()).thenReturn(() -> realmId); + int threadCount = 10; + List threads = new ArrayList<>(); + ConcurrentLinkedQueue exceptions = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue allEvents = new ConcurrentLinkedQueue<>(); + + for (int i = 0; i < threadCount; i++) { + Thread t = + new Thread( + () -> { + try { + for (int j = 0; j < 10; j++) { + PolarisEvent event = createSampleEvent(); + allEvents.add(event); + eventListener.processEvent(event); + } + } catch (Exception e) { + exceptions.add(e); + } + }); + threads.add(t); + } + + // Start all threads + threads.forEach(Thread::start); + // Wait for all threads to finish + for (Thread t : threads) { + t.join(); + } + // There should be no exceptions + if (!exceptions.isEmpty()) { + throw new AssertionError("Exceptions occurred in concurrent checkAndFlushBufferIfNecessary: ", exceptions.peek()); + } + + ArgumentCaptor> eventsCaptor = ArgumentCaptor.forClass(List.class); + verify(polarisMetaStoreManager, atLeastOnce()).writeEvents(any(), eventsCaptor.capture()); + Awaitility.await("expected amount of records should be processed") + .atMost(Duration.ofSeconds(30)) + .pollDelay(Duration.ofMillis(500)) + .pollInterval(Duration.ofMillis(500)) + .untilAsserted( + () -> { + List eventsProcessed = eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); + assertThat(eventsProcessed.size()).isEqualTo(allEvents.size()); + }); + List seenEvents = eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); + assertThat(seenEvents.size()).isEqualTo(allEvents.size()); + assertThat(seenEvents).hasSameElementsAs(allEvents); } @Test @@ -177,16 +225,7 @@ public void testRequestIdFunctionalityWithContainerRequestContext() { when(mockContainerRequestContext.hasProperty("requestId")).thenReturn(true); when(mockContainerRequestContext.getProperty("requestId")).thenReturn(expectedRequestId); - // Use reflection to set the containerRequestContext field - try { - java.lang.reflect.Field field = - InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField( - "containerRequestContext"); - field.setAccessible(true); - field.set(eventListener, mockContainerRequestContext); - } catch (Exception e) { - throw new RuntimeException("Failed to set containerRequestContext field", e); - } + eventListener.containerRequestContext = mockContainerRequestContext; String actualRequestId = eventListener.getRequestId(); assertThat(actualRequestId) @@ -230,16 +269,7 @@ public void testRequestIdFunctionalityWithContainerRequestContextButNoProperty() ContainerRequestContext mockContainerRequestContext = Mockito.mock(ContainerRequestContext.class); when(mockContainerRequestContext.hasProperty("requestId")).thenReturn(false); - - try { - java.lang.reflect.Field field = - InMemoryBufferPolarisPersistenceEventListener.class.getDeclaredField( - "containerRequestContext"); - field.setAccessible(true); - field.set(eventListener, mockContainerRequestContext); - } catch (Exception e) { - throw new RuntimeException("Failed to set containerRequestContext field", e); - } + eventListener.containerRequestContext = mockContainerRequestContext; String requestId = eventListener.getRequestId(); @@ -260,9 +290,9 @@ private List addEventsWithoutTriggeringFlush(String realmId) { RealmContext realmContext = () -> realmId; when(callContext.getRealmContext()).thenReturn(realmContext); for (PolarisEvent realmEvent : realmEvents) { - eventListener.addToBuffer(realmEvent); + eventListener.processEvent(realmEvent); } - verify(polarisMetaStoreManager, times(0)).writeEvents(Mockito.any(), Mockito.any()); + verify(polarisMetaStoreManager, times(0)).writeEvents(any(), any()); return realmEvents; } From b83f7480586fb78eb2bd422756ed06d8588af386 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Fri, 29 Aug 2025 00:13:21 -0700 Subject: [PATCH 12/17] spotlessapply --- .../core/persistence/AtomicOperationMetaStoreManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java index e5bc0f918b..f6f04143f0 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/AtomicOperationMetaStoreManager.java @@ -45,7 +45,6 @@ import org.apache.polaris.core.entity.PolarisEntityId; import org.apache.polaris.core.entity.PolarisEntitySubType; import org.apache.polaris.core.entity.PolarisEntityType; -import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.core.entity.PolarisGrantRecord; import org.apache.polaris.core.entity.PolarisPrincipalSecrets; import org.apache.polaris.core.entity.PolarisPrivilege; From a51446d304bed52d1a40e8366f0e3e12f476d802 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Fri, 29 Aug 2025 00:30:02 -0700 Subject: [PATCH 13/17] spotlessapply again --- .../service/admin/PolarisServiceImpl.java | 3 +- .../catalog/iceberg/IcebergCatalog.java | 30 +++++++---- .../iceberg/IcebergCatalogHandler.java | 4 +- .../events/AfterCatalogCreatedEvent.java | 3 +- .../events/AfterTableCommitedEvent.java | 8 +-- .../events/AfterTableCreatedEvent.java | 2 +- .../events/AfterTableRefreshedEvent.java | 2 +- .../events/AfterTaskAttemptedEvent.java | 3 +- .../events/AfterViewCommitedEvent.java | 9 ++-- .../events/BeforeTableCommitedEvent.java | 3 +- .../events/BeforeTableRefreshedEvent.java | 2 +- .../events/BeforeTaskAttemptedEvent.java | 3 +- .../events/BeforeViewCommitedEvent.java | 9 ++-- .../events/BeforeViewRefreshedEvent.java | 2 +- ...currentLinkedQueueWithApproximateSize.java | 36 ++++++------- ...BufferPolarisPersistenceEventListener.java | 18 ++++--- .../ratelimiter/RateLimiterFilter.java | 4 +- .../service/task/TaskExecutorImpl.java | 4 +- .../iceberg/AbstractIcebergCatalogTest.java | 3 +- ...erPolarisPersistenceEventListenerTest.java | 53 +++++++++---------- 20 files changed, 109 insertions(+), 92 deletions(-) diff --git a/runtime/service/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java b/runtime/service/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java index da468734e2..217aa43f4e 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/admin/PolarisServiceImpl.java @@ -172,8 +172,7 @@ public Response createCatalog( validateExternalCatalog(catalog); Catalog newCatalog = CatalogEntity.of(adminService.createCatalog(request)).asCatalog(); LOGGER.info("Created new catalog {}", newCatalog); - polarisEventListener.onAfterCatalogCreated( - new AfterCatalogCreatedEvent(newCatalog.getName())); + polarisEventListener.onAfterCatalogCreated(new AfterCatalogCreatedEvent(newCatalog.getName())); return Response.status(Response.Status.CREATED).build(); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java index 57b0d8be7d..a86cd40df9 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java @@ -140,7 +140,6 @@ import org.apache.polaris.service.events.BeforeTableRefreshedEvent; import org.apache.polaris.service.events.BeforeViewCommitedEvent; import org.apache.polaris.service.events.BeforeViewRefreshedEvent; -import org.apache.polaris.service.events.PolarisEvent; import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.task.TaskExecutor; import org.apache.polaris.service.types.NotificationRequest; @@ -1412,10 +1411,14 @@ public void doRefresh() { } public void doCommit(TableMetadata base, TableMetadata metadata) { - polarisEventListener.onBeforeTableCommited(new BeforeTableCommitedEvent(tableIdentifier, base, metadata)); + polarisEventListener.onBeforeTableCommited( + new BeforeTableCommitedEvent(tableIdentifier, base, metadata)); LOGGER.debug( - "doCommit for table {} with metadataBefore {}, metadataAfter {}", tableIdentifier, base, metadata); + "doCommit for table {} with metadataBefore {}, metadataAfter {}", + tableIdentifier, + base, + metadata); // TODO: Maybe avoid writing metadata if there's definitely a transaction conflict if (null == base && !namespaceExists(tableIdentifier.namespace())) { throw new NoSuchNamespaceException( @@ -1553,7 +1556,8 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { updateTableLike(tableIdentifier, entity); } - polarisEventListener.onAfterTableCommited(new AfterTableCommitedEvent(catalogName, tableIdentifier, base, metadata)); + polarisEventListener.onAfterTableCommited( + new AfterTableCommitedEvent(catalogName, tableIdentifier, base, metadata)); } @Override @@ -1744,7 +1748,8 @@ public void doRefresh() { if (latestLocation == null) { disableRefresh(); } else { - polarisEventListener.onBeforeViewRefreshed(new BeforeViewRefreshedEvent(catalogName, identifier)); + polarisEventListener.onBeforeViewRefreshed( + new BeforeViewRefreshedEvent(catalogName, identifier)); refreshFromMetadataLocation( latestLocation, SHOULD_RETRY_REFRESH_PREDICATE, @@ -1766,15 +1771,21 @@ public void doRefresh() { return ViewMetadataParser.read(fileIO.newInputFile(metadataLocation)); }); - polarisEventListener.onAfterViewRefreshed(new AfterViewRefreshedEvent(catalogName, identifier)); + polarisEventListener.onAfterViewRefreshed( + new AfterViewRefreshedEvent(catalogName, identifier)); } } public void doCommit(ViewMetadata base, ViewMetadata metadata) { - polarisEventListener.onBeforeViewCommited(new BeforeViewCommitedEvent(catalogName, identifier, base, metadata)); + polarisEventListener.onBeforeViewCommited( + new BeforeViewCommitedEvent(catalogName, identifier, base, metadata)); // TODO: Maybe avoid writing metadata if there's definitely a transaction conflict - LOGGER.debug("doCommit for view {} with metadataBefore {}, metadataAfter {}", identifier, base, metadata); + LOGGER.debug( + "doCommit for view {} with metadataBefore {}, metadataAfter {}", + identifier, + base, + metadata); if (null == base && !namespaceExists(identifier.namespace())) { throw new NoSuchNamespaceException( "Cannot create view '%s'. Namespace does not exist: '%s'", @@ -1867,7 +1878,8 @@ public void doCommit(ViewMetadata base, ViewMetadata metadata) { updateTableLike(identifier, entity); } - polarisEventListener.onAfterViewCommited(new AfterViewCommitedEvent(catalogName, identifier, base, metadata)); + polarisEventListener.onAfterViewCommited( + new AfterViewCommitedEvent(catalogName, identifier, base, metadata)); } protected String writeNewMetadataIfRequired(ViewMetadata metadata) { diff --git a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java index c6b3b0c4ad..fe40fcdec2 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandler.java @@ -99,7 +99,6 @@ import org.apache.polaris.service.config.ReservedProperties; import org.apache.polaris.service.context.catalog.CallContextCatalogFactory; import org.apache.polaris.service.events.AfterTableCreatedEvent; -import org.apache.polaris.service.events.PolarisEvent; import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.http.IcebergHttpUtil; import org.apache.polaris.service.http.IfNoneMatch; @@ -389,7 +388,8 @@ public LoadTableResponse createTableDirect(Namespace namespace, CreateTableReque .build(); LoadTableResponse resp = catalogHandlerUtils.createTable(baseCatalog, namespace, requestWithoutReservedProperties); - polarisEventListener.onAfterTableCreated(new AfterTableCreatedEvent(catalogName, identifier, resp.tableMetadata())); + polarisEventListener.onAfterTableCreated( + new AfterTableCreatedEvent(catalogName, identifier, resp.tableMetadata())); return resp; } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java index 740a3d5ea3..0ed7cef48b 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterCatalogCreatedEvent.java @@ -20,5 +20,4 @@ package org.apache.polaris.service.events; /** Emitted Polaris creates a catalog. */ -public record AfterCatalogCreatedEvent(String catalogName) - implements PolarisEvent {} +public record AfterCatalogCreatedEvent(String catalogName) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java index ee744659cb..a69d304483 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCommitedEvent.java @@ -26,10 +26,10 @@ * while committing. * * @param catalogName The catalog name where this table exists. - * @param identifier The identifier. - * @param base The old metadata. - * @param metadata The new metadata. + * @param identifier The identifier. + * @param base The old metadata. + * @param metadata The new metadata. */ public record AfterTableCommitedEvent( - String catalogName, TableIdentifier identifier, TableMetadata base, TableMetadata metadata) + String catalogName, TableIdentifier identifier, TableMetadata base, TableMetadata metadata) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java index 7bfb5f3b4e..dd8939a358 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableCreatedEvent.java @@ -24,5 +24,5 @@ /** Emitted when Polaris creates a table. */ public record AfterTableCreatedEvent( - String catalogName, TableIdentifier identifier, TableMetadata metadata) + String catalogName, TableIdentifier identifier, TableMetadata metadata) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java index d79ab9ed6d..caef8e3c53 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTableRefreshedEvent.java @@ -23,7 +23,7 @@ /** * Emitted after Polaris refreshes its known version of a table's metadata by fetching the latest. * - * @param catalogName The name of the catalog where the table is located + * @param catalogName The name of the catalog where the table is located * @param tableIdentifier The identifier of the table that was refreshed. */ public record AfterTableRefreshedEvent(String catalogName, TableIdentifier tableIdentifier) diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java index 2068504434..d1ccfb1960 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterTaskAttemptedEvent.java @@ -26,4 +26,5 @@ * initial (non-retried) attempt starts counting from 1. * @param success Whether the attempt succeeded. */ -public record AfterTaskAttemptedEvent(long taskEntityId, int attempt, boolean success) implements PolarisEvent {} +public record AfterTaskAttemptedEvent(long taskEntityId, int attempt, boolean success) + implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java index 86ff8192b4..a7d01ecc29 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/AfterViewCommitedEvent.java @@ -26,9 +26,10 @@ * while committing. * * @param catalogName The catalog name where the view is located - * @param identifier The identifier. - * @param base The old metadata. - * @param metadata The new metadata. + * @param identifier The identifier. + * @param base The old metadata. + * @param metadata The new metadata. */ -public record AfterViewCommitedEvent(String catalogName, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) +public record AfterViewCommitedEvent( + String catalogName, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java index 0e98c06337..511f6f0976 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableCommitedEvent.java @@ -30,5 +30,6 @@ * @param metadataBefore The old metadata. * @param metadataAfter The new metadata. */ -public record BeforeTableCommitedEvent(TableIdentifier tableIdentifier, TableMetadata metadataBefore, TableMetadata metadataAfter) +public record BeforeTableCommitedEvent( + TableIdentifier tableIdentifier, TableMetadata metadataBefore, TableMetadata metadataAfter) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java index 0ed57e59ca..f5daade51d 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTableRefreshedEvent.java @@ -24,7 +24,7 @@ * Emitted when Polaris intends to refresh its known version of a table's metadata by fetching the * latest. * - * @param catalogName The name of the catalog where the view is located. + * @param catalogName The name of the catalog where the view is located. * @param tableIdentifier The identifier of the table being refreshed. */ public record BeforeTableRefreshedEvent(String catalogName, TableIdentifier tableIdentifier) diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java index 7580ff81cd..fcfa4400af 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeTaskAttemptedEvent.java @@ -25,5 +25,4 @@ * @param attempt The attempt number. Each retry of the task will have its own attempt number. The * initial (non-retried) attempt starts counting from 1. */ -public record BeforeTaskAttemptedEvent(long taskEntityId, int attempt) - implements PolarisEvent {} +public record BeforeTaskAttemptedEvent(long taskEntityId, int attempt) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java index a86a07fc68..c9303de71f 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewCommitedEvent.java @@ -27,9 +27,10 @@ * fail Polaris-side validation checks. * * @param catalogName The name of the catalog where the view is located. - * @param identifier The identifier. - * @param base The old metadata. - * @param metadata The new metadata. + * @param identifier The identifier. + * @param base The old metadata. + * @param metadata The new metadata. */ -public record BeforeViewCommitedEvent(String catalogName, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) +public record BeforeViewCommitedEvent( + String catalogName, TableIdentifier identifier, ViewMetadata base, ViewMetadata metadata) implements PolarisEvent {} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java index 559e5754a4..32b3250c72 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/BeforeViewRefreshedEvent.java @@ -24,7 +24,7 @@ * Emitted when Polaris intends to refresh its known version of a view's metadata by fetching the * latest. * - * @param catalogName The name of the catalog where the view is located. + * @param catalogName The name of the catalog where the view is located. * @param viewIdentifier The identifier of the view being refreshed. */ public record BeforeViewRefreshedEvent(String catalogName, TableIdentifier viewIdentifier) diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java index d24bcb0e55..b8b05a8341 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java @@ -24,27 +24,27 @@ import java.util.stream.Stream; public class ConcurrentLinkedQueueWithApproximateSize { - private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - private final AtomicInteger size = new AtomicInteger(); + private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private final AtomicInteger size = new AtomicInteger(); - public void add(T event) { - queue.add(event); - size.getAndIncrement(); - } + public void add(T event) { + queue.add(event); + size.getAndIncrement(); + } - public boolean isEmpty() { - return queue.isEmpty(); - } + public boolean isEmpty() { + return queue.isEmpty(); + } - public T peek() { - return queue.peek(); - } + public T peek() { + return queue.peek(); + } - public int size() { - return size.get(); - } + public int size() { + return size.get(); + } - public Stream stream() { - return queue.stream(); - } + public Stream stream() { + return queue.stream(); + } } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java index fb1cb27801..f1d87da933 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListener.java @@ -36,7 +36,6 @@ import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.PolarisDiagnostics; import org.apache.polaris.core.context.CallContext; @@ -57,8 +56,8 @@ public class InMemoryBufferPolarisPersistenceEventListener extends PolarisPersis private static final String REQUEST_ID_KEY = "requestId"; private final MetaStoreManagerFactory metaStoreManagerFactory; - private final ConcurrentHashMap> buffer = - new ConcurrentHashMap<>(); + private final ConcurrentHashMap> + buffer = new ConcurrentHashMap<>(); private final ScheduledExecutorService executor; private final Set> futures = ConcurrentHashMap.newKeySet(); private final Duration timeToFlush; @@ -144,7 +143,8 @@ String getRequestId() { void processEvent(PolarisEvent polarisEvent) { String realmId = callContext.getRealmContext().getRealmIdentifier(); - ConcurrentLinkedQueueWithApproximateSize realmQueue = buffer.computeIfAbsent(realmId, k -> new ConcurrentLinkedQueueWithApproximateSize<>()); + ConcurrentLinkedQueueWithApproximateSize realmQueue = + buffer.computeIfAbsent(realmId, k -> new ConcurrentLinkedQueueWithApproximateSize<>()); realmQueue.add(polarisEvent); if (realmQueue.size() >= maxBufferSize) { futures.add(executor.submit(() -> checkAndFlushBufferIfNecessary(realmId, true))); @@ -167,16 +167,20 @@ void checkAndFlushBufferIfNecessary(String realmId, boolean forceFlush) { if (elapsed.compareTo(timeToFlush) > 0 || queue.size() >= maxBufferSize || forceFlush) { // Atomically replace old queue with new queue - boolean replaced = buffer.replace(realmId, queue, new ConcurrentLinkedQueueWithApproximateSize<>()); + boolean replaced = + buffer.replace(realmId, queue, new ConcurrentLinkedQueueWithApproximateSize<>()); if (!replaced) { // Another thread concurrently modified the buffer, so do not continue return; } RealmContext realmContext = () -> realmId; - PolarisMetaStoreManager metaStoreManager = metaStoreManagerFactory.getOrCreateMetaStoreManager(realmContext); + PolarisMetaStoreManager metaStoreManager = + metaStoreManagerFactory.getOrCreateMetaStoreManager(realmContext); BasePersistence basePersistence = metaStoreManagerFactory.getOrCreateSession(realmContext); - metaStoreManager.writeEvents(new PolarisCallContext(realmContext, basePersistence, polarisDiagnostics), queue.stream().toList()); + metaStoreManager.writeEvents( + new PolarisCallContext(realmContext, basePersistence, polarisDiagnostics), + queue.stream().toList()); } } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java b/runtime/service/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java index c1c825b587..61d27a0407 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/ratelimiter/RateLimiterFilter.java @@ -29,7 +29,6 @@ import java.io.IOException; import org.apache.polaris.service.config.FilterPriorities; import org.apache.polaris.service.events.BeforeRequestRateLimitedEvent; -import org.apache.polaris.service.events.PolarisEvent; import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,7 +55,8 @@ public RateLimiterFilter(RateLimiter rateLimiter, PolarisEventListener polarisEv public void filter(ContainerRequestContext ctx) throws IOException { if (!rateLimiter.canProceed()) { polarisEventListener.onBeforeRequestRateLimited( - new BeforeRequestRateLimitedEvent(ctx.getMethod(), ctx.getUriInfo().getAbsolutePath().toString())); + new BeforeRequestRateLimitedEvent( + ctx.getMethod(), ctx.getUriInfo().getAbsolutePath().toString())); ctx.abortWith(Response.status(Response.Status.TOO_MANY_REQUESTS).build()); LOGGER.atDebug().log("Rate limiting request"); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java b/runtime/service/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java index 25d0584d7f..d425c16754 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/task/TaskExecutorImpl.java @@ -45,7 +45,6 @@ import org.apache.polaris.core.persistence.PolarisMetaStoreManager; import org.apache.polaris.service.events.AfterTaskAttemptedEvent; import org.apache.polaris.service.events.BeforeTaskAttemptedEvent; -import org.apache.polaris.service.events.PolarisEvent; import org.apache.polaris.service.events.listeners.PolarisEventListener; import org.apache.polaris.service.tracing.TracingFilter; import org.slf4j.Logger; @@ -187,7 +186,8 @@ protected void handleTask(long taskEntityId, CallContext ctx, int attempt) { .log("Unable to execute async task"); } } finally { - polarisEventListener.onAfterTaskAttempted(new AfterTaskAttemptedEvent(taskEntityId, attempt, success)); + polarisEventListener.onAfterTaskAttempted( + new AfterTaskAttemptedEvent(taskEntityId, attempt, success)); } } diff --git a/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/AbstractIcebergCatalogTest.java b/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/AbstractIcebergCatalogTest.java index ce15bd3c35..e237231982 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/AbstractIcebergCatalogTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/catalog/iceberg/AbstractIcebergCatalogTest.java @@ -2232,7 +2232,8 @@ public void testEventsAreEmitted() { var beforeTableEvent = testPolarisEventListener.getLatest(BeforeTableCommitedEvent.class); Assertions.assertThat(beforeTableEvent.tableIdentifier()).isEqualTo(TestData.TABLE); - Assertions.assertThat(beforeTableEvent.metadataBefore().properties().get(key)).isEqualTo(valOld); + Assertions.assertThat(beforeTableEvent.metadataBefore().properties().get(key)) + .isEqualTo(valOld); Assertions.assertThat(beforeTableEvent.metadataAfter().properties().get(key)).isEqualTo(valNew); var afterTableEvent = testPolarisEventListener.getLatest(AfterTableCommitedEvent.class); diff --git a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index d6ff90f895..db67139c03 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -22,7 +22,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.ArgumentMatchers.refEq; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -38,8 +37,6 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.stream.Collectors; - import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.context.RealmContext; @@ -97,8 +94,7 @@ public void testProcessEventFlushesAfterConfiguredTime() { // Push clock forwards to flush the buffer clock.add(CONFIG_TIME_TO_FLUSH_IN_MS.multipliedBy(2)); eventListener.checkAndFlushBufferIfNecessary(realmId, false); - verify(polarisMetaStoreManager, times(1)) - .writeEvents(any(), eq(eventsAddedToBuffer)); + verify(polarisMetaStoreManager, times(1)).writeEvents(any(), eq(eventsAddedToBuffer)); } @Test @@ -173,18 +169,18 @@ public void testProcessEventIsThreadSafe() throws Exception { for (int i = 0; i < threadCount; i++) { Thread t = - new Thread( - () -> { - try { - for (int j = 0; j < 10; j++) { - PolarisEvent event = createSampleEvent(); - allEvents.add(event); - eventListener.processEvent(event); - } - } catch (Exception e) { - exceptions.add(e); - } - }); + new Thread( + () -> { + try { + for (int j = 0; j < 10; j++) { + PolarisEvent event = createSampleEvent(); + allEvents.add(event); + eventListener.processEvent(event); + } + } catch (Exception e) { + exceptions.add(e); + } + }); threads.add(t); } @@ -196,21 +192,24 @@ public void testProcessEventIsThreadSafe() throws Exception { } // There should be no exceptions if (!exceptions.isEmpty()) { - throw new AssertionError("Exceptions occurred in concurrent checkAndFlushBufferIfNecessary: ", exceptions.peek()); + throw new AssertionError( + "Exceptions occurred in concurrent checkAndFlushBufferIfNecessary: ", exceptions.peek()); } ArgumentCaptor> eventsCaptor = ArgumentCaptor.forClass(List.class); verify(polarisMetaStoreManager, atLeastOnce()).writeEvents(any(), eventsCaptor.capture()); Awaitility.await("expected amount of records should be processed") - .atMost(Duration.ofSeconds(30)) - .pollDelay(Duration.ofMillis(500)) - .pollInterval(Duration.ofMillis(500)) - .untilAsserted( - () -> { - List eventsProcessed = eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); - assertThat(eventsProcessed.size()).isEqualTo(allEvents.size()); - }); - List seenEvents = eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); + .atMost(Duration.ofSeconds(30)) + .pollDelay(Duration.ofMillis(500)) + .pollInterval(Duration.ofMillis(500)) + .untilAsserted( + () -> { + List eventsProcessed = + eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); + assertThat(eventsProcessed.size()).isEqualTo(allEvents.size()); + }); + List seenEvents = + eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); assertThat(seenEvents.size()).isEqualTo(allEvents.size()); assertThat(seenEvents).hasSameElementsAs(allEvents); } From 16d606c8ce158a31117f62520afb5f0801734076 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Fri, 29 Aug 2025 01:10:01 -0700 Subject: [PATCH 14/17] isolate sensitive test --- .../InMemoryBufferPolarisPersistenceEventListenerTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index db67139c03..4fa4cff344 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -46,6 +46,8 @@ import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; +import org.junit.jupiter.api.parallel.ExecutionMode; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.threeten.extra.MutableClock; @@ -158,6 +160,7 @@ public void testCheckAndFlushBufferIfNecessaryIsThreadSafe() throws Exception { verify(polarisMetaStoreManager, times(1)).writeEvents(any(), eq(events)); } + @Execution(ExecutionMode.SAME_THREAD) @Test public void testProcessEventIsThreadSafe() throws Exception { String realmId = "realm1"; @@ -193,7 +196,7 @@ public void testProcessEventIsThreadSafe() throws Exception { // There should be no exceptions if (!exceptions.isEmpty()) { throw new AssertionError( - "Exceptions occurred in concurrent checkAndFlushBufferIfNecessary: ", exceptions.peek()); + "Exceptions occurred in concurrent processEvent: ", exceptions.peek()); } ArgumentCaptor> eventsCaptor = ArgumentCaptor.forClass(List.class); @@ -206,7 +209,7 @@ public void testProcessEventIsThreadSafe() throws Exception { () -> { List eventsProcessed = eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); - assertThat(eventsProcessed.size()).isEqualTo(allEvents.size()); + assertThat(eventsProcessed.size()).isGreaterThanOrEqualTo(allEvents.size()); }); List seenEvents = eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); From 076dee897312bd7fb55e10f8cbf4b950348cbc00 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Fri, 29 Aug 2025 14:14:15 -0700 Subject: [PATCH 15/17] Revision based on review from @adutra --- .../listeners/ConcurrentLinkedQueueWithApproximateSize.java | 2 +- .../InMemoryBufferPolarisPersistenceEventListenerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java index b8b05a8341..3be25e1ad2 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/ConcurrentLinkedQueueWithApproximateSize.java @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Stream; -public class ConcurrentLinkedQueueWithApproximateSize { +class ConcurrentLinkedQueueWithApproximateSize { private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); private final AtomicInteger size = new AtomicInteger(); diff --git a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index 4fa4cff344..f5f961b031 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -199,7 +199,7 @@ public void testProcessEventIsThreadSafe() throws Exception { "Exceptions occurred in concurrent processEvent: ", exceptions.peek()); } - ArgumentCaptor> eventsCaptor = ArgumentCaptor.forClass(List.class); + ArgumentCaptor> eventsCaptor = ArgumentCaptor.captor(); verify(polarisMetaStoreManager, atLeastOnce()).writeEvents(any(), eventsCaptor.capture()); Awaitility.await("expected amount of records should be processed") .atMost(Duration.ofSeconds(30)) From 6d8290232d2ab720a5c0a614aa627b810ce6f4cb Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Sat, 30 Aug 2025 23:41:15 -0700 Subject: [PATCH 16/17] remove extra import statement --- runtime/service/build.gradle.kts | 1 - 1 file changed, 1 deletion(-) diff --git a/runtime/service/build.gradle.kts b/runtime/service/build.gradle.kts index 472b640fb8..18204b3578 100644 --- a/runtime/service/build.gradle.kts +++ b/runtime/service/build.gradle.kts @@ -166,7 +166,6 @@ dependencies { testFixturesImplementation(libs.jakarta.enterprise.cdi.api) testFixturesImplementation(libs.jakarta.annotation.api) testFixturesImplementation(libs.jakarta.ws.rs.api) - testFixturesApi(libs.threeten.extra) testFixturesImplementation(platform(libs.quarkus.bom)) testFixturesImplementation("io.quarkus:quarkus-rest-client") From 217af0b1a0033e031474c139004499ccce561d34 Mon Sep 17 00:00:00 2001 From: adnanhemani Date: Tue, 2 Sep 2025 18:46:21 -0700 Subject: [PATCH 17/17] spotlessapply --- .../InMemoryBufferPolarisPersistenceEventListenerTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java index 2776945545..026e741d40 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/InMemoryBufferPolarisPersistenceEventListenerTest.java @@ -209,7 +209,8 @@ public void testProcessEventIsThreadSafe() throws Exception { () -> { clock.add(500, ChronoUnit.MILLIS); ArgumentCaptor> eventsCaptor = ArgumentCaptor.captor(); - verify(polarisMetaStoreManager, atLeastOnce()).writeEvents(any(), eventsCaptor.capture()); + verify(polarisMetaStoreManager, atLeastOnce()) + .writeEvents(any(), eventsCaptor.capture()); List eventsProcessed = eventsCaptor.getAllValues().stream().flatMap(List::stream).toList(); if (eventsProcessed.size() > 100) {