From 208d55a64d914f370a7350eb10081ebe1b6d6781 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Thu, 30 Apr 2026 21:35:45 -0400 Subject: [PATCH 01/15] feat: pre-delete dependency guard for DROP TABLE MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Refuses a DROP TABLE while an active Pipeline still references the resource (as either source or sink), so dropping the underlying Kafka topic / Venice store / MySQL table can't silently orphan a downstream pipeline. Validator framework, made Connection-aware: - Validated.validate(Issues, Connection) (was: validate(Issues)) - ValidatorProvider.validators(T, Connection) (was: validators(T)) - ValidationService.validate(T, Issues, Connection) - ValidationService.validateOrThrow(T, Connection) - ValidationService.validateOrThrow(Collection, Connection) - ValidationService.validators(T, Connection) PendingDelete wrapper (hoptimator-api): - Explicit "this is being deleted" signal so unrelated callers of validateOrThrow(source, connection) don't accidentally trigger pre-delete checks. - Carries an optional selfOwnerUid so cascade-deleted children can be excluded from the dependent set. K8s indexed lookup: - PipelineDependencyLabels stamps `depends-on-` labels on every Pipeline CRD at create time, naming each source/sink. The slug is a 16-char SHA-256 prefix of `_`; an annotation lists the full identifiers so a slug collision can be detected at check time. - PipelineDependencyChecker uses a server-indexed label-selector list + annotation cross-check + selfOwnerUid filter. - K8sPipelineDeployer threads sources/sink through and calls PipelineDependencyLabels.labelsFor / annotationFor at toK8sObject(). K8sPipelineBundle and K8sMaterializedViewDeployer pass the data through. Dispatch: - K8sValidatorProvider returns a K8sPipelineDependencyValidator for PendingDelete; registered via META-INF/services/com.linkedin.hoptimator.ValidatorProvider. - K8sPipelineDependencyValidator wraps PipelineDependencyChecker as a Validator. DROP TABLE wiring: - HoptimatorDdlExecutor calls ValidationService.validateOrThrow(new PendingDelete<>(source), connection) before DeploymentService.delete in the table branch. HoptimatorDdlUtils.removeTableFromSchema() is the symmetric inverse of registerTemporaryTableInSchema() for cleanup. Implementor side-effects (no behavior change): - KafkaDeployer / VeniceDeployer / MySqlDeployer no longer need a declarative DependencyGuarded marker — the guard fires from the validator framework before delete() is reached. - All existing Validated implementors (DefaultValidator, CompatibilityValidatorBase, AvroTableValidator, K8sViewTable) and ValidatorProvider implementors (DefaultValidatorProvider, CompatibilityValidatorProvider, AvroValidatorProvider) updated to the new signatures. Tests: PipelineDependencyLabelsTest, PipelineDependencyCheckerTest, K8sPipelineDeployerTest assertions for stamping, validator-framework test updates throughout. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../linkedin/hoptimator/PendingDelete.java | 44 +++++ .../com/linkedin/hoptimator/Validated.java | 11 +- .../com/linkedin/hoptimator/Validator.java | 5 +- .../hoptimator/ValidatorProvider.java | 9 +- .../linkedin/hoptimator/ValidatorTest.java | 4 +- .../hoptimator/avro/AvroTableValidator.java | 3 +- .../avro/AvroValidatorProvider.java | 3 +- .../avro/AvroTableValidatorTest.java | 8 +- .../avro/AvroValidatorProviderTest.java | 6 +- .../jdbc/CompatibilityValidatorBase.java | 4 +- .../jdbc/CompatibilityValidatorProvider.java | 3 +- .../jdbc/DefaultValidatorProvider.java | 3 +- .../jdbc/HoptimatorDdlExecutor.java | 26 ++- .../hoptimator/jdbc/HoptimatorDdlUtils.java | 51 ++++- .../hoptimator/jdbc/ValidationService.java | 30 +-- .../jdbc/CompatibilityValidatorBaseTest.java | 8 +- .../CompatibilityValidatorProviderTest.java | 6 +- .../jdbc/DefaultValidatorProviderTest.java | 14 +- .../jdbc/HoptimatorDdlExecutorTest.java | 4 +- .../jdbc/HoptimatorDdlUtilsTest.java | 93 +++++++++ .../hoptimator/jdbc/TestSqlScripts.java | 5 +- .../jdbc/ValidationServiceTest.java | 18 +- .../jdbc/ValidatorProviderTest.java | 5 +- .../k8s/K8sMaterializedViewDeployer.java | 20 +- .../hoptimator/k8s/K8sPipelineBundle.java | 17 +- .../k8s/K8sPipelineDependencyValidator.java | 40 ++++ .../hoptimator/k8s/K8sPipelineDeployer.java | 47 ++++- .../hoptimator/k8s/K8sValidatorProvider.java | 36 ++++ .../linkedin/hoptimator/k8s/K8sViewTable.java | 3 +- .../k8s/PipelineDependencyChecker.java | 111 +++++++++++ .../k8s/PipelineDependencyLabels.java | 144 ++++++++++++++ .../com.linkedin.hoptimator.ValidatorProvider | 1 + .../k8s/K8sMaterializedViewDeployerTest.java | 6 +- .../hoptimator/k8s/K8sPipelineBundleTest.java | 9 +- .../k8s/K8sPipelineDeployerTest.java | 62 ++++++ .../hoptimator/k8s/K8sViewTableTest.java | 8 +- .../k8s/PipelineDependencyCheckerTest.java | 137 +++++++++++++ .../k8s/PipelineDependencyLabelsTest.java | 184 ++++++++++++++++++ .../hoptimator/kafka/KafkaDeployer.java | 3 +- .../hoptimator/kafka/KafkaDeployerTest.java | 2 +- .../hoptimator/mysql/MySqlDeployer.java | 10 +- .../hoptimator/mysql/MySqlDeployerTest.java | 27 +-- .../util/DeploymentServiceTest.java | 2 +- .../hoptimator/venice/VeniceDeployer.java | 3 +- .../hoptimator/venice/VeniceDeployerTest.java | 16 +- 45 files changed, 1114 insertions(+), 137 deletions(-) create mode 100644 hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java create mode 100644 hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java create mode 100644 hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java create mode 100644 hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java create mode 100644 hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java create mode 100644 hoptimator-k8s/src/main/resources/META-INF/services/com.linkedin.hoptimator.ValidatorProvider create mode 100644 hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java create mode 100644 hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java new file mode 100644 index 00000000..d4d77201 --- /dev/null +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java @@ -0,0 +1,44 @@ +package com.linkedin.hoptimator; + +import java.util.Objects; + + +/** + * A type-tagged wrapper signalling that {@code target} is about to be deleted. Pre-delete + * validators (e.g. dependency guards that block DROP TABLE when a pipeline still references + * the resource) key off this wrapper rather than the raw target type — so an unrelated future + * caller of {@code ValidationService.validateOrThrow(source, connection)} doesn't accidentally + * trigger delete-intent checks. + * + *

An optional {@code selfOwnerUid} lets the caller declare an "umbrella" K8s resource UID + * whose owned objects should be excluded from the dependent set — e.g. a LogicalTable CRD's UID, + * so its child Pipeline CRDs (which reference tier sources by SQL) don't self-block the drop. + */ +public final class PendingDelete { + + private final T target; + private final String selfOwnerUid; + + public PendingDelete(T target) { + this(target, null); + } + + public PendingDelete(T target, String selfOwnerUid) { + this.target = Objects.requireNonNull(target, "target"); + this.selfOwnerUid = selfOwnerUid; + } + + public T target() { + return target; + } + + /** UID of the K8s resource whose owned objects should be excluded from the dependent set. */ + public String selfOwnerUid() { + return selfOwnerUid; + } + + @Override + public String toString() { + return "PendingDelete[" + target + (selfOwnerUid != null ? ", self=" + selfOwnerUid : "") + "]"; + } +} diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validated.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validated.java index fae19bcf..12d21537 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validated.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validated.java @@ -1,6 +1,15 @@ package com.linkedin.hoptimator; +import java.sql.Connection; + + public interface Validated { - void validate(Validator.Issues issues); + /** + * Validates {@code this}, recording any problems in {@code issues}. The connection is always + * supplied so validators can run lookups against external systems (e.g. pre-delete dependency + * checks). Pass {@code null} only when the caller genuinely has no connection — most validators + * ignore it, but some require it. + */ + void validate(Validator.Issues issues, Connection connection); } diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validator.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validator.java index 82e72831..9db6a279 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validator.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validator.java @@ -1,5 +1,6 @@ package com.linkedin.hoptimator; +import java.sql.Connection; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -52,8 +53,8 @@ public DefaultValidator(T t) { } @Override - public void validate(Issues issues) { - t.validate(issues.child(t.getClass().getSimpleName())); + public void validate(Issues issues, Connection connection) { + t.validate(issues.child(t.getClass().getSimpleName()), connection); } } diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java index 55b60a44..5a4b4d7f 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java @@ -1,9 +1,16 @@ package com.linkedin.hoptimator; +import java.sql.Connection; import java.util.Collection; public interface ValidatorProvider { - Collection validators(T obj); + /** + * Returns validators that should be applied to {@code obj}. The connection is always supplied + * — providers that don't need it can ignore it (matches the {@code DeployerProvider} pattern). + * Implementations capturing the connection in returned validators must accept that the + * connection may be {@code null} when the caller has none. + */ + Collection validators(T obj, Connection connection); } diff --git a/hoptimator-api/src/test/java/com/linkedin/hoptimator/ValidatorTest.java b/hoptimator-api/src/test/java/com/linkedin/hoptimator/ValidatorTest.java index ddd98438..ab1f4719 100644 --- a/hoptimator-api/src/test/java/com/linkedin/hoptimator/ValidatorTest.java +++ b/hoptimator-api/src/test/java/com/linkedin/hoptimator/ValidatorTest.java @@ -184,10 +184,10 @@ void testCheckClosedMessageContainsPath() { @Test void testDefaultValidatorDelegatesToTarget() { - Validated target = issues -> issues.error("target error"); + Validated target = (issues, conn) -> issues.error("target error"); Validator.DefaultValidator validator = new Validator.DefaultValidator<>(target); Validator.Issues issues = new Validator.Issues("root"); - validator.validate(issues); + validator.validate(issues, null); assertFalse(issues.valid()); } diff --git a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroTableValidator.java b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroTableValidator.java index 8d9f45eb..57d35fed 100644 --- a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroTableValidator.java +++ b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroTableValidator.java @@ -17,6 +17,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.OutputStream; +import java.sql.Connection; /** Validates that tables follow Avro schema evolution rules. */ @@ -29,7 +30,7 @@ class AvroTableValidator implements Validator { } @Override - public void validate(Issues issues) { + public void validate(Issues issues, Connection connection) { try { CalciteSchema originalSchema = schema.unwrap(CalciteSchema.class); if (originalSchema == null || originalSchema.schema == null) { diff --git a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroValidatorProvider.java b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroValidatorProvider.java index 0f4c3ef7..de5a986b 100644 --- a/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroValidatorProvider.java +++ b/hoptimator-avro/src/main/java/com/linkedin/hoptimator/avro/AvroValidatorProvider.java @@ -4,6 +4,7 @@ import com.linkedin.hoptimator.ValidatorProvider; import org.apache.calcite.schema.SchemaPlus; +import java.sql.Connection; import java.util.Collection; import java.util.Collections; @@ -12,7 +13,7 @@ public class AvroValidatorProvider implements ValidatorProvider { @Override - public Collection validators(T obj) { + public Collection validators(T obj, Connection connection) { if (obj instanceof SchemaPlus) { return Collections.singletonList(new AvroTableValidator((SchemaPlus) obj)); } else { diff --git a/hoptimator-avro/src/test/java/com/linkedin/hoptimator/avro/AvroTableValidatorTest.java b/hoptimator-avro/src/test/java/com/linkedin/hoptimator/avro/AvroTableValidatorTest.java index 0d10c71e..b843bf96 100644 --- a/hoptimator-avro/src/test/java/com/linkedin/hoptimator/avro/AvroTableValidatorTest.java +++ b/hoptimator-avro/src/test/java/com/linkedin/hoptimator/avro/AvroTableValidatorTest.java @@ -41,7 +41,7 @@ void testValidateCatchesClassCastExceptionSilently() { AvroTableValidator validator = new AvroTableValidator(schema); Validator.Issues issues = new Validator.Issues("test"); - validator.validate(issues); + validator.validate(issues, null); assertTrue(issues.valid(), "ClassCastException should be silently caught"); } @@ -53,7 +53,7 @@ void testValidateThrowsForNullOriginalSchema() { AvroTableValidator validator = new AvroTableValidator(schema); Validator.Issues issues = new Validator.Issues("test"); - assertThrows(IllegalArgumentException.class, () -> validator.validate(issues)); + assertThrows(IllegalArgumentException.class, () -> validator.validate(issues, null)); } @Test @@ -99,7 +99,7 @@ protected Map getTableMap() { AvroTableValidator validator = new AvroTableValidator(schema); Validator.Issues issues = new Validator.Issues("root"); - validator.validate(issues); + validator.validate(issues, null); assertFalse(issues.valid(), "Incompatible schema (INT→VARCHAR) should produce validation errors"); @@ -137,7 +137,7 @@ protected Map getTableMap() { AvroTableValidator validator = new AvroTableValidator(schema); Validator.Issues issues = new Validator.Issues("root"); - validator.validate(issues); + validator.validate(issues, null); assertTrue(issues.valid(), "Compatible schemas should pass validation without errors"); } diff --git a/hoptimator-avro/src/test/java/com/linkedin/hoptimator/avro/AvroValidatorProviderTest.java b/hoptimator-avro/src/test/java/com/linkedin/hoptimator/avro/AvroValidatorProviderTest.java index 6ce8ba25..0bb41746 100644 --- a/hoptimator-avro/src/test/java/com/linkedin/hoptimator/avro/AvroValidatorProviderTest.java +++ b/hoptimator-avro/src/test/java/com/linkedin/hoptimator/avro/AvroValidatorProviderTest.java @@ -24,7 +24,7 @@ class AvroValidatorProviderTest { void testValidatorsReturnsAvroTableValidatorForSchemaPlus() { AvroValidatorProvider provider = new AvroValidatorProvider(); - Collection validators = provider.validators(schemaPlus); + Collection validators = provider.validators(schemaPlus, null); assertEquals(1, validators.size()); assertInstanceOf(AvroTableValidator.class, validators.iterator().next()); @@ -34,7 +34,7 @@ void testValidatorsReturnsAvroTableValidatorForSchemaPlus() { void testValidatorsReturnsEmptyForNonSchemaPlus() { AvroValidatorProvider provider = new AvroValidatorProvider(); - Collection validators = provider.validators("not-a-schema"); + Collection validators = provider.validators("not-a-schema", null); assertTrue(validators.isEmpty()); } @@ -43,7 +43,7 @@ void testValidatorsReturnsEmptyForNonSchemaPlus() { void testValidatorsReturnsEmptyForNull() { AvroValidatorProvider provider = new AvroValidatorProvider(); - Collection validators = provider.validators(null); + Collection validators = provider.validators(null, null); assertTrue(validators.isEmpty()); } diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorBase.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorBase.java index 44a1de00..c6a8b12f 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorBase.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorBase.java @@ -6,6 +6,8 @@ import org.apache.calcite.schema.Table; import org.apache.calcite.schema.lookup.LikePattern; +import java.sql.Connection; + /** Base class for shared schema evolution validators. */ abstract class CompatibilityValidatorBase implements Validator { @@ -17,7 +19,7 @@ abstract class CompatibilityValidatorBase implements Validator { } @Override - public void validate(Issues issues) { + public void validate(Issues issues, Connection connection) { try { CalciteSchema originalSchema = schema.unwrap(CalciteSchema.class); if (originalSchema == null || originalSchema.schema == null) { diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorProvider.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorProvider.java index 46579b8b..32588782 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorProvider.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorProvider.java @@ -4,6 +4,7 @@ import com.linkedin.hoptimator.ValidatorProvider; import org.apache.calcite.schema.SchemaPlus; +import java.sql.Connection; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -13,7 +14,7 @@ public class CompatibilityValidatorProvider implements ValidatorProvider { @Override - public Collection validators(T obj) { + public Collection validators(T obj, Connection connection) { if (obj instanceof SchemaPlus) { return Arrays.asList(new Validator[]{new BackwardCompatibilityValidator((SchemaPlus) obj), new ForwardCompatibilityValidator((SchemaPlus) obj)}); diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/DefaultValidatorProvider.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/DefaultValidatorProvider.java index 8461b26e..fb21346e 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/DefaultValidatorProvider.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/DefaultValidatorProvider.java @@ -4,6 +4,7 @@ import com.linkedin.hoptimator.Validator; import com.linkedin.hoptimator.ValidatorProvider; +import java.sql.Connection; import java.util.Collection; import java.util.Collections; @@ -12,7 +13,7 @@ public class DefaultValidatorProvider implements ValidatorProvider { @Override - public Collection validators(T obj) { + public Collection validators(T obj, Connection connection) { if (obj instanceof Validated) { return Collections.singletonList(new Validator.DefaultValidator<>((Validated) obj)); } else { diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index 678de93b..d1574563 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -19,7 +19,9 @@ */ package com.linkedin.hoptimator.jdbc; +import com.linkedin.hoptimator.Database; import com.linkedin.hoptimator.Deployer; +import com.linkedin.hoptimator.PendingDelete; import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.Trigger; import com.linkedin.hoptimator.UserJob; @@ -107,7 +109,7 @@ public DdlExecutor getDdlExecutor() { public void execute(SqlCreateView create, CalcitePrepare.Context context) { logger.info("Validating statement: {}", create); try { - ValidationService.validateOrThrow(create); + ValidationService.validateOrThrow(create, connection); } catch (SQLException e) { throw new DdlException(create, e.getMessage(), e); } @@ -145,9 +147,9 @@ public void execute(SqlCreateView create, CalcitePrepare.Context context) { Collection deployers = null; try { logger.info("Validating deployable resources for view {}", viewName); - ValidationService.validateOrThrow(viewTable); + ValidationService.validateOrThrow(viewTable, connection); deployers = DeploymentService.deployers(view, connection); - ValidationService.validateOrThrow(deployers); + ValidationService.validateOrThrow(deployers, connection); logger.info("Validated view {}", viewName); if (create.getReplace()) { logger.info("Deploying update view {}", viewName); @@ -195,7 +197,7 @@ public void execute(SqlCreateMaterializedView create, CalcitePrepare.Context con public void execute(SqlCreateTrigger create, CalcitePrepare.Context context) { logger.info("Validating statement: {}", create); try { - ValidationService.validateOrThrow(create); + ValidationService.validateOrThrow(create, connection); } catch (SQLException e) { throw new DdlException(create, e.getMessage(), e); } @@ -233,9 +235,9 @@ public void execute(SqlCreateTrigger create, CalcitePrepare.Context context) { Collection deployers = null; try { logger.info("Validating trigger {} with deployers", name); - ValidationService.validateOrThrow(trigger); + ValidationService.validateOrThrow(trigger, connection); deployers = DeploymentService.deployers(trigger, connection); - ValidationService.validateOrThrow(deployers); + ValidationService.validateOrThrow(deployers, connection); logger.info("Validated trigger {}", name); if (create.getReplace()) { logger.info("Updating trigger {}", name); @@ -296,7 +298,7 @@ public void execute(SqlResumeTrigger resume, CalcitePrepare.Context context) { public void execute(SqlDropTrigger drop, CalcitePrepare.Context context) { logger.info("Validating statement: {}", drop); try { - ValidationService.validateOrThrow(drop); + ValidationService.validateOrThrow(drop, connection); } catch (SQLException e) { throw new DdlException(drop, e.getMessage(), e); } @@ -331,7 +333,7 @@ public void execute(SqlDropTrigger drop, CalcitePrepare.Context context) { private void updateTriggerPausedState(SqlNode sqlNode, SqlIdentifier triggerName, boolean paused) { logger.info("Validating statement: {}", sqlNode); try { - ValidationService.validateOrThrow(sqlNode); + ValidationService.validateOrThrow(sqlNode, connection); } catch (SQLException e) { throw new DdlException(sqlNode, e.getMessage(), e); } @@ -366,7 +368,7 @@ private void updateTriggerPausedState(SqlNode sqlNode, SqlIdentifier triggerName public void execute(SqlDropObject drop, CalcitePrepare.Context context) { logger.info("Validating statement: {}", drop); try { - ValidationService.validateOrThrow(drop); + ValidationService.validateOrThrow(drop, connection); } catch (SQLException e) { throw new DdlException(drop, e.getMessage(), e); } @@ -394,6 +396,8 @@ public void execute(SqlDropObject drop, CalcitePrepare.Context context) { final List schemaPath = pair.left.path(null); List tablePath = new ArrayList<>(schemaPath); tablePath.add(tableName); + String database = pair.left.schema instanceof Database + ? ((Database) pair.left.schema).databaseName() : null; Collection deployers = null; try { @@ -435,6 +439,10 @@ public void execute(SqlDropObject drop, CalcitePrepare.Context context) { TemporaryTable temporaryTable = (TemporaryTable) table; source = new Source(temporaryTable.databaseName(), tablePath, Collections.emptyMap()); } + // Pre-delete dependency guard. PendingDelete is the explicit "delete intent" signal + // — only validators that key off it (the K8s dep checker) fire here. The check throws + // before any deployer-level state change. + ValidationService.validateOrThrow(new PendingDelete<>(source), connection); deployers = DeploymentService.deployers(source, connection); logger.info("Deleting table {}", tableName); DeploymentService.delete(deployers); diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlUtils.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlUtils.java index 6ff1e551..5e29f047 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlUtils.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlUtils.java @@ -296,7 +296,7 @@ static SpecifyResult processCreateMaterializedView(CalcitePrepare.Context ctx, HoptimatorConnection.HoptimatorConnectionDualLogger logger = conn.getLogger(HoptimatorDdlUtils.class); // Validate the DDL statement. logger.info("Validating statement: {}", create); - ValidationService.validateOrThrow(create); + ValidationService.validateOrThrow(create, conn); // Extract query SQL (rename columns if a column list was provided) and plan the query. // This is done first — before schema/conflict checks — so that: @@ -374,10 +374,10 @@ static SpecifyResult processCreateMaterializedView(CalcitePrepare.Context ctx, // Validate the hook and its deployers. logger.info("Validating materialized view {}", viewName); - ValidationService.validateOrThrow(hook); + ValidationService.validateOrThrow(hook, conn); deployers = DeploymentService.deployers(hook, conn); logger.info("Validating deployable resources for materialized view {}", viewName); - ValidationService.validateOrThrow(deployers); + ValidationService.validateOrThrow(deployers, conn); logger.info("Validated materialized view {}", viewName); // Execute (create/update) or collect specs (specify). @@ -442,7 +442,7 @@ static SpecifyResult processCreateTable(CalcitePrepare.Context ctx, HoptimatorCo HoptimatorConnection.HoptimatorConnectionDualLogger logger = conn.getLogger(HoptimatorDdlUtils.class); logger.info("Validating statement: {}", create); - ValidationService.validateOrThrow(create); + ValidationService.validateOrThrow(create, conn); // TODO: Add support for populating new tables from a query as a one-time operation. if (create.query != null) { @@ -573,10 +573,10 @@ public RexNode newColumnDefaultValue(RelOptTable table, int iColumn, boolean success = false; try { logger.info("Validating new table {}", source); - ValidationService.validateOrThrow(source); + ValidationService.validateOrThrow(source, conn); deployers = DeploymentService.deployers(source, conn); logger.info("Validating deployable resources for table {}", tableName); - ValidationService.validateOrThrow(deployers); + ValidationService.validateOrThrow(deployers, conn); if (mode == DdlMode.UPDATE) { logger.info("Deploying update table {}", source); @@ -638,7 +638,7 @@ static SpecifyResult processCreateDatabase(HoptimatorConnection conn, HoptimatorConnection.HoptimatorConnectionDualLogger logger = conn.getLogger(HoptimatorDdlUtils.class); logger.info("Validating statement: {}", create); - ValidationService.validateOrThrow(create); + ValidationService.validateOrThrow(create, conn); if (create.name.names.size() > 1) { throw new SQLException("Database names cannot be compound identifiers."); @@ -651,9 +651,9 @@ static SpecifyResult processCreateDatabase(HoptimatorConnection conn, Collection deployers = null; try { logger.info("Validating database {}", name); - ValidationService.validateOrThrow(database); + ValidationService.validateOrThrow(database, conn); deployers = DeploymentService.deployers(database, conn); - ValidationService.validateOrThrow(deployers); + ValidationService.validateOrThrow(deployers, conn); List specs = mode.executeDeployers(deployers, conn); if (mode.mutable()) { @@ -892,4 +892,37 @@ public static Runnable registerTemporaryTableInSchema(HoptimatorConnection conn, return registerTemporaryTable(rootSchema, tableName, rowType, databaseName); } } + + /** + * Removes {@code tableName} from the tier schema identified by {@code (catalog, schema)}, the + * inverse of {@link #registerTemporaryTableInSchema}. Silent no-op if the catalog, schema or + * table entry does not exist (e.g. the connection was opened after the table was already gone) + * — remove must never fail a user-visible DROP that has already succeeded at the backend. + */ + public static void removeTableFromSchema(HoptimatorConnection conn, + @Nullable String catalog, @Nullable String schema, String tableName) { + if (conn == null) { + return; + } + SchemaPlus rootSchema = conn.calciteConnection().getRootSchema(); + SchemaPlus target; + if (catalog != null) { + SchemaPlus catalogSchemaPlus = rootSchema.subSchemas().get(catalog); + if (catalogSchemaPlus == null) { + return; + } + if (schema == null) { + target = catalogSchemaPlus; + } else { + target = catalogSchemaPlus.subSchemas().get(schema); + } + } else if (schema != null) { + target = rootSchema.subSchemas().get(schema); + } else { + target = rootSchema; + } + if (target != null && target.tables().get(tableName) != null) { + target.removeTable(tableName); + } + } } diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ValidationService.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ValidationService.java index 72fa2e0b..ccd6ceb5 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ValidationService.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/ValidationService.java @@ -28,40 +28,40 @@ public static Validator.Issues validate(Connection connection) { } CalciteConnection conn = (CalciteConnection) connection; Validator.Issues issues = new Validator.Issues(""); - walk(conn.getRootSchema(), issues); + walk(conn.getRootSchema(), issues, connection); return issues; } - private static void walk(SchemaPlus schema, Validator.Issues issues) { - validate(schema, issues); + private static void walk(SchemaPlus schema, Validator.Issues issues, Connection connection) { + validate(schema, issues, connection); for (String x : schema.subSchemas().getNames(LikePattern.any())) { - walk(schema.subSchemas().get(x), issues.child(x)); + walk(schema.subSchemas().get(x), issues.child(x), connection); } for (String x : schema.tables().getNames(LikePattern.any())) { - walk(schema.tables().get(x), issues.child(x)); + walk(schema.tables().get(x), issues.child(x), connection); } } - private static void walk(Table table, Validator.Issues issues) { - validate(table, issues); + private static void walk(Table table, Validator.Issues issues, Connection connection) { + validate(table, issues, connection); } - public static void validate(T obj, Validator.Issues issues) { - validators(obj).forEach(x -> x.validate(issues)); + public static void validate(T obj, Validator.Issues issues, Connection connection) { + validators(obj, connection).forEach(x -> x.validate(issues, connection)); } - public static void validateOrThrow(T obj) throws SQLException { + public static void validateOrThrow(T obj, Connection connection) throws SQLException { Validator.Issues issues = new Validator.Issues(""); - validate(obj, issues); + validate(obj, issues, connection); if (!issues.valid()) { throw new SQLDataException("Failed validation:\n" + issues); } } - public static void validateOrThrow(Collection objs) throws SQLException { + public static void validateOrThrow(Collection objs, Connection connection) throws SQLException { Validator.Issues issues = new Validator.Issues(""); for (T obj : objs) { - validate(obj, issues); + validate(obj, issues, connection); if (!issues.valid()) { throw new SQLDataException("Failed validation:\n" + issues); } @@ -75,7 +75,7 @@ public static Collection providers() { return providers; } - public static Collection validators(T obj) { - return providers().stream().flatMap(x -> x.validators(obj).stream()).collect(Collectors.toList()); + public static Collection validators(T obj, Connection connection) { + return providers().stream().flatMap(x -> x.validators(obj, connection).stream()).collect(Collectors.toList()); } } diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorBaseTest.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorBaseTest.java index 07dd51b4..b8ec8c23 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorBaseTest.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorBaseTest.java @@ -38,7 +38,7 @@ protected void validate(Table table, Table originalTable, Validator.Issues issue }; Validator.Issues issues = new Validator.Issues("test"); - validator.validate(issues); + validator.validate(issues, null); assertTrue(issues.valid()); } @@ -57,7 +57,7 @@ protected void validate(Table table, Table originalTable, Validator.Issues issue Validator.Issues issues = new Validator.Issues("test"); try { - validator.validate(issues); + validator.validate(issues, null); } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("Null original schema")); } @@ -99,7 +99,7 @@ protected void validate(Table table, Table originalTable, Validator.Issues issue }; Validator.Issues issues = new Validator.Issues("test"); - validator.validate(issues); + validator.validate(issues, null); assertTrue(issues.valid()); } @@ -125,7 +125,7 @@ protected void validate(Table table, Table originalTable, Validator.Issues issue }; Validator.Issues issues = new Validator.Issues("test"); - validator.validate(issues); + validator.validate(issues, null); // brandNewTable won't have an original, so validate should be skipped assertTrue(issues.valid()); diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorProviderTest.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorProviderTest.java index db2f3627..2fc0b8a4 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorProviderTest.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/CompatibilityValidatorProviderTest.java @@ -23,7 +23,7 @@ class CompatibilityValidatorProviderTest { void testValidatorsReturnsTwoValidatorsForSchemaPlus() { CompatibilityValidatorProvider provider = new CompatibilityValidatorProvider(); - Collection validators = provider.validators(mockSchema); + Collection validators = provider.validators(mockSchema, null); assertEquals(2, validators.size()); } @@ -32,7 +32,7 @@ void testValidatorsReturnsTwoValidatorsForSchemaPlus() { void testValidatorsReturnsEmptyForNonSchemaPlus() { CompatibilityValidatorProvider provider = new CompatibilityValidatorProvider(); - Collection validators = provider.validators("not-a-schema"); + Collection validators = provider.validators("not-a-schema", null); assertTrue(validators.isEmpty()); } @@ -41,7 +41,7 @@ void testValidatorsReturnsEmptyForNonSchemaPlus() { void testValidatorsReturnsEmptyForNull() { CompatibilityValidatorProvider provider = new CompatibilityValidatorProvider(); - Collection validators = provider.validators(null); + Collection validators = provider.validators(null, null); assertTrue(validators.isEmpty()); } diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/DefaultValidatorProviderTest.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/DefaultValidatorProviderTest.java index ece661c8..f838f51f 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/DefaultValidatorProviderTest.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/DefaultValidatorProviderTest.java @@ -15,9 +15,9 @@ class DefaultValidatorProviderTest { @Test void testValidatorsReturnsValidatorForValidatedObject() { DefaultValidatorProvider provider = new DefaultValidatorProvider(); - Validated validated = issues -> issues.error("test error"); + Validated validated = (issues, conn) -> issues.error("test error"); - Collection validators = provider.validators(validated); + Collection validators = provider.validators(validated, null); assertEquals(1, validators.size()); } @@ -26,7 +26,7 @@ void testValidatorsReturnsValidatorForValidatedObject() { void testValidatorsReturnsEmptyForNonValidatedObject() { DefaultValidatorProvider provider = new DefaultValidatorProvider(); - Collection validators = provider.validators("not-validated"); + Collection validators = provider.validators("not-validated", null); assertTrue(validators.isEmpty()); } @@ -35,7 +35,7 @@ void testValidatorsReturnsEmptyForNonValidatedObject() { void testValidatorsReturnsEmptyForNull() { DefaultValidatorProvider provider = new DefaultValidatorProvider(); - Collection validators = provider.validators(null); + Collection validators = provider.validators(null, null); assertTrue(validators.isEmpty()); } @@ -43,11 +43,11 @@ void testValidatorsReturnsEmptyForNull() { @Test void testReturnedValidatorDelegates() { DefaultValidatorProvider provider = new DefaultValidatorProvider(); - Validated validated = issues -> issues.error("validation failed"); + Validated validated = (issues, conn) -> issues.error("validation failed"); - Collection validators = provider.validators(validated); + Collection validators = provider.validators(validated, null); Validator.Issues issues = new Validator.Issues("test"); - validators.iterator().next().validate(issues); + validators.iterator().next().validate(issues, null); assertTrue(issues.toString().contains("validation failed")); } diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutorTest.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutorTest.java index 4ac1507f..a902cccf 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutorTest.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutorTest.java @@ -1221,10 +1221,10 @@ private void addMaterializedViewToDefaultSchema(CalcitePrepare.Context context, private void stubValidationToFail() { // Casting to Object in the lambda forces Java's overload resolution to pick the T-object // overload (erased to Object) rather than the Collection overload. - mockValidationService.when(() -> ValidationService.validateOrThrow(any(Object.class))) + mockValidationService.when(() -> ValidationService.validateOrThrow(any(Object.class), any())) .thenThrow(new SQLException("validation failed")); // Also stub the Collection overload for callers that pass deployer collections. - mockValidationService.when(() -> ValidationService.validateOrThrow(any(Collection.class))) + mockValidationService.when(() -> ValidationService.validateOrThrow(any(Collection.class), any())) .thenThrow(new SQLException("validation failed")); } } diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlUtilsTest.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlUtilsTest.java index 4013e367..29a11973 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlUtilsTest.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlUtilsTest.java @@ -44,6 +44,7 @@ import java.util.Map; import java.util.Properties; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -1604,4 +1605,96 @@ public String databaseName() { return name; } } + + // ---- removeTableFromSchema tests ---- + + @Test + void removeTableFromSchemaToleratesNullConnection() { + // Must not NPE — this overload is called from code paths that may lack a connection. + assertDoesNotThrow(() -> HoptimatorDdlUtils.removeTableFromSchema(null, null, null, "ANY")); + } + + @Test + void removeTableFromSchemaRemovesFromRootWhenNoSchema() throws SQLException { + HoptimatorDriver driver = new HoptimatorDriver(); + try (HoptimatorConnection connection = + (HoptimatorConnection) driver.connect("jdbc:hoptimator://catalogs=util", new Properties())) { + RelDataTypeFactory typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + RelDataType rowType = typeFactory.createStructType(Collections.emptyList(), Collections.emptyList()); + HoptimatorDdlUtils.registerTemporaryTableInSchema( + connection, null, null, "ROOT_TMP", rowType, "db"); + SchemaPlus rootSchema = connection.calciteConnection().getRootSchema(); + assertNotNull(rootSchema.tables().get("ROOT_TMP")); + + HoptimatorDdlUtils.removeTableFromSchema(connection, null, null, "ROOT_TMP"); + + assertEquals(null, rootSchema.tables().get("ROOT_TMP")); + } + } + + @Test + void removeTableFromSchemaRemovesFromTierSchema() throws SQLException { + HoptimatorDriver driver = new HoptimatorDriver(); + try (HoptimatorConnection connection = + (HoptimatorConnection) driver.connect("jdbc:hoptimator://catalogs=util", new Properties())) { + SchemaPlus rootSchema = connection.calciteConnection().getRootSchema(); + rootSchema.add("KAFKA", new AbstractSchema()); + RelDataTypeFactory typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + RelDataType rowType = typeFactory.createStructType(Collections.emptyList(), Collections.emptyList()); + HoptimatorDdlUtils.registerTemporaryTableInSchema( + connection, null, "KAFKA", "my_topic", rowType, "kafka-db"); + SchemaPlus tierSchema = rootSchema.subSchemas().get("KAFKA"); + assertNotNull(tierSchema); + assertNotNull(tierSchema.tables().get("my_topic")); + + HoptimatorDdlUtils.removeTableFromSchema(connection, null, "KAFKA", "my_topic"); + + assertEquals(null, tierSchema.tables().get("my_topic")); + } + } + + @Test + void removeTableFromSchemaRemovesFromCatalogAndSchema() throws SQLException { + HoptimatorDriver driver = new HoptimatorDriver(); + try (HoptimatorConnection connection = + (HoptimatorConnection) driver.connect("jdbc:hoptimator://catalogs=util", new Properties())) { + SchemaPlus rootSchema = connection.calciteConnection().getRootSchema(); + SchemaPlus catalogSchema = rootSchema.add("CAT", new AbstractSchema()); + SchemaPlus dbSchema = catalogSchema.add("DB", new AbstractSchema()); + RelDataTypeFactory typeFactory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + RelDataType rowType = typeFactory.createStructType(Collections.emptyList(), Collections.emptyList()); + HoptimatorDdlUtils.registerTemporaryTableInSchema( + connection, "CAT", "DB", "t", rowType, "mysql-db"); + assertNotNull(dbSchema.tables().get("t")); + + HoptimatorDdlUtils.removeTableFromSchema(connection, "CAT", "DB", "t"); + + assertEquals(null, dbSchema.tables().get("t")); + } + } + + @Test + void removeTableFromSchemaIsNoOpWhenEntriesMissing() throws SQLException { + HoptimatorDriver driver = new HoptimatorDriver(); + try (HoptimatorConnection connection = + (HoptimatorConnection) driver.connect("jdbc:hoptimator://catalogs=util", new Properties())) { + // Missing catalog, missing schema, missing table — each path must be silent. + SchemaPlus rootSchema = connection.calciteConnection().getRootSchema(); + + assertDoesNotThrow(() -> + HoptimatorDdlUtils.removeTableFromSchema(connection, "MISSING_CATALOG", "S", "t")); + assertDoesNotThrow(() -> + HoptimatorDdlUtils.removeTableFromSchema(connection, null, "MISSING_SCHEMA", "t")); + assertDoesNotThrow(() -> + HoptimatorDdlUtils.removeTableFromSchema(connection, null, null, "MISSING_TABLE")); + + // None of those calls should have accidentally created the missing catalog / schema. + assertNull(rootSchema.subSchemas().get("MISSING_CATALOG"), + "missing-catalog call must not create the catalog"); + assertNull(rootSchema.subSchemas().get("MISSING_SCHEMA"), + "missing-schema call must not create the schema"); + assertNull(rootSchema.tables().get("MISSING_TABLE"), + "missing-table call must not create the table"); + } + } } diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/TestSqlScripts.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/TestSqlScripts.java index 9c417e65..d735543d 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/TestSqlScripts.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/TestSqlScripts.java @@ -13,6 +13,7 @@ import java.net.URLClassLoader; import java.nio.file.Files; import java.nio.file.Path; +import java.sql.Connection; import java.util.Collection; import java.util.List; @@ -76,7 +77,7 @@ private void useTestValidators() throws IOException { public static class CreateViewValidatorProvider implements ValidatorProvider { @Override - public Collection validators(T obj) { + public Collection validators(T obj, Connection connection) { if (obj instanceof SqlCreateView || obj instanceof SqlCreateMaterializedView) { return List.of(new SqlCreateViewValidator()); } @@ -88,7 +89,7 @@ static class SqlCreateViewValidator implements Validator { static final String ERROR_MESSAGE = "Create view is not allowed in this test."; @Override - public void validate(Issues issues) { + public void validate(Issues issues, Connection connection) { issues.error(ERROR_MESSAGE); } } diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/ValidationServiceTest.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/ValidationServiceTest.java index 52383b04..af2bead6 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/ValidationServiceTest.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/ValidationServiceTest.java @@ -103,7 +103,7 @@ void testValidateConnectionCallsWalk() throws SQLException { void testValidateObjIssuesInvokesValidators() { ValidatorProviderTest.enableErrors(); Validator.Issues issues = new Validator.Issues("test"); - ValidationService.validate("any-object", issues); + ValidationService.validate("any-object", issues, null); assertFalse(issues.valid(), "validate(obj, issues) must invoke validators; if forEach is removed no error fires"); assertTrue(issues.toString().contains("injected error"), @@ -115,7 +115,7 @@ void testValidateObjIssuesInvokesValidators() { void testValidateOrThrowSingleObjectThrowsWhenErrorRecorded() { ValidatorProviderTest.enableErrors(); assertThrows(SQLException.class, - () -> ValidationService.validateOrThrow("any-object"), + () -> ValidationService.validateOrThrow("any-object", null), "validateOrThrow must throw when a provider records an error"); } @@ -125,27 +125,27 @@ void testValidateOrThrowSingleObjectThrowsWhenErrorRecorded() { @Test void testValidateOrThrowSingleObjectPassesWhenValid() throws SQLException { // ValidatorProviderTest is in no-error mode (reset in setUp) - ValidationService.validateOrThrow("test-object"); + ValidationService.validateOrThrow("test-object", null); } @Test void testValidateOrThrowCollectionThrowsWhenErrorRecorded() { ValidatorProviderTest.enableErrors(); assertThrows(SQLException.class, - () -> ValidationService.validateOrThrow(Arrays.asList("obj1", "obj2")), + () -> ValidationService.validateOrThrow(Arrays.asList("obj1", "obj2"), null), "validateOrThrow(Collection) must throw when provider records an error"); } @Test void testValidateOrThrowCollectionPassesWhenValid() throws SQLException { Collection validObjects = Arrays.asList("obj1", "obj2"); - ValidationService.validateOrThrow(validObjects); + ValidationService.validateOrThrow(validObjects, null); } @Test void testValidateOrThrowCollectionPassesWithEmptyCollection() throws SQLException { Collection emptyCollection = Collections.emptyList(); - ValidationService.validateOrThrow(emptyCollection); + ValidationService.validateOrThrow(emptyCollection, null); } // ValidatorProviderTest is registered via META-INF/services so ServiceLoader must find it. @@ -161,7 +161,7 @@ void testProvidersReturnsAtLeastOneRegisteredProvider() { @Test void testValidatorsReturnsValidatorsFromRegisteredProvider() { ValidatorProviderTest.enableErrors(); - Collection validators = ValidationService.validators("any-object"); + Collection validators = ValidationService.validators("any-object", null); assertNotNull(validators); assertFalse(validators.isEmpty(), "validators() must return the non-empty list provided by ValidatorProviderTest"); @@ -175,14 +175,14 @@ void testProvidersReturnsCollection() { @Test void testValidatorsReturnsCollection() { - Collection validators = ValidationService.validators("test-object"); + Collection validators = ValidationService.validators("test-object", null); assertNotNull(validators); } @Test void testValidatePopulatesIssues() { Validator.Issues issues = new Validator.Issues("test"); - ValidationService.validate("test-object", issues); + ValidationService.validate("test-object", issues, null); assertNotNull(issues); } diff --git a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/ValidatorProviderTest.java b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/ValidatorProviderTest.java index c8319bc0..9537c4fb 100644 --- a/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/ValidatorProviderTest.java +++ b/hoptimator-jdbc/src/test/java/com/linkedin/hoptimator/jdbc/ValidatorProviderTest.java @@ -3,6 +3,7 @@ import com.linkedin.hoptimator.Validator; import com.linkedin.hoptimator.ValidatorProvider; +import java.sql.Connection; import java.util.Collection; import java.util.Collections; import java.util.concurrent.atomic.AtomicBoolean; @@ -39,10 +40,10 @@ static Object lastSeen() { } @Override - public Collection validators(T obj) { + public Collection validators(T obj, Connection connection) { LAST_SEEN.set(obj); if (SHOULD_ERROR.get()) { - return Collections.singletonList(issues -> issues.error("ValidatorProviderTest injected error")); + return Collections.singletonList((issues, conn) -> issues.error("ValidatorProviderTest injected error")); } return Collections.emptyList(); } diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployer.java index 60d657b9..7cd3ebff 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployer.java @@ -2,6 +2,7 @@ import com.linkedin.hoptimator.Deployer; import com.linkedin.hoptimator.MaterializedView; +import com.linkedin.hoptimator.Sink; import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.SqlDialect; import com.linkedin.hoptimator.util.DeploymentService; @@ -9,10 +10,17 @@ import java.sql.SQLException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; -/** Deploys View and Pipeline objects, along with all the pipeline elements. */ +/** + * Deploys View and Pipeline objects, along with all the pipeline elements. + * + *

The dependency guard is implemented on {@link K8sViewDeployer} (which is what gets + * routed to during DROP MV), so this class doesn't need to implement + * {@link com.linkedin.hoptimator.DependencyGuarded} itself. + */ class K8sMaterializedViewDeployer implements Deployer { private final MaterializedView view; @@ -34,8 +42,8 @@ K8sViewDeployer createViewDeployer(MaterializedView view, K8sContext context) { } K8sPipelineBundle createPipelineBundle(String name, List pipelineSpecs, String sql, - K8sContext viewContext) { - return new K8sPipelineBundle(name, pipelineSpecs, sql, viewContext); + Collection sources, Sink sink, K8sContext viewContext) { + return new K8sPipelineBundle(name, pipelineSpecs, sql, sources, sink, viewContext); } @Override @@ -45,7 +53,8 @@ public void create() throws SQLException { List pipelineSpecs = pipelineSpecs(); V1OwnerReference viewRef = viewDeployer.createAndReference(); K8sContext viewContext = context.withOwner(viewRef); - K8sPipelineBundle bundle = createPipelineBundle(name, pipelineSpecs, sql(), viewContext); + K8sPipelineBundle bundle = createPipelineBundle(name, pipelineSpecs, sql(), + view.pipeline().sources(), view.pipeline().sink(), viewContext); deployers.add(bundle); bundle.create(); } @@ -58,7 +67,8 @@ public void update() throws SQLException { List pipelineSpecs = pipelineSpecs(); V1OwnerReference viewRef = viewDeployer.updateAndReference(); K8sContext viewContext = context.withOwner(viewRef); - K8sPipelineBundle bundle = createPipelineBundle(name, pipelineSpecs, sql(), viewContext); + K8sPipelineBundle bundle = createPipelineBundle(name, pipelineSpecs, sql(), + view.pipeline().sources(), view.pipeline().sink(), viewContext); deployers.add(bundle); bundle.update(); } diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineBundle.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineBundle.java index e2b1606b..8fe455d2 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineBundle.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineBundle.java @@ -2,11 +2,14 @@ import java.sql.SQLException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import io.kubernetes.client.openapi.models.V1OwnerReference; import com.linkedin.hoptimator.Deployer; +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; /** @@ -25,16 +28,22 @@ public class K8sPipelineBundle implements Deployer { private final K8sPipelineDeployer pipelineDeployer; private final List deployers = new ArrayList<>(); + /** + * {@code sources} and {@code sink} are stamped as {@code depends-on-*} + * labels on the Pipeline CRD so the delete-time guard in {@link PipelineDependencyChecker} + * can find this pipeline by label selector. + */ public K8sPipelineBundle(String name, List pipelineSpecs, String sql, - K8sContext ownerContext) { + Collection sources, Sink sink, K8sContext ownerContext) { this.name = name; this.pipelineSpecs = pipelineSpecs; this.ownerContext = ownerContext; - this.pipelineDeployer = createPipelineDeployer(name, pipelineSpecs, sql, ownerContext); + this.pipelineDeployer = createPipelineDeployer(name, pipelineSpecs, sql, sources, sink, ownerContext); } - K8sPipelineDeployer createPipelineDeployer(String n, List specs, String sql, K8sContext ctx) { - return new K8sPipelineDeployer(n, specs, sql, ctx); + K8sPipelineDeployer createPipelineDeployer(String n, List specs, String sql, + Collection sources, Sink sink, K8sContext ctx) { + return new K8sPipelineDeployer(n, specs, sql, sources, sink, ctx); } K8sYamlDeployerImpl createYamlDeployerImpl(K8sContext pipelineContext, List specs) { diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java new file mode 100644 index 00000000..b27e490c --- /dev/null +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java @@ -0,0 +1,40 @@ +package com.linkedin.hoptimator.k8s; + +import java.sql.Connection; +import java.sql.SQLException; + +import com.linkedin.hoptimator.Source; +import com.linkedin.hoptimator.Validator; + + +/** + * Pre-delete dependency check, run by the validator framework when a {@link Source} is wrapped + * in {@link com.linkedin.hoptimator.PendingDelete}. Delegates to the existing + * {@link PipelineDependencyChecker} (label-selector + annotation collision-guard + self-uid + * exclusion) and surfaces any blocking pipeline as a validation error rather than throwing + * directly — that's what the validator contract calls for. + */ +final class K8sPipelineDependencyValidator implements Validator { + + private final Source source; + private final String selfOwnerUid; + + K8sPipelineDependencyValidator(Source source, String selfOwnerUid) { + this.source = source; + this.selfOwnerUid = selfOwnerUid; + } + + @Override + public void validate(Issues issues, Connection connection) { + if (connection == null) { + issues.error("Cannot run pre-delete dependency check without a connection"); + return; + } + try { + PipelineDependencyChecker.assertNoExternalDependents( + K8sContext.create(connection), source.database(), source.path(), selfOwnerUid); + } catch (SQLException e) { + issues.error(e.getMessage()); + } + } +} diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java index 5e76bb19..bd70ae22 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java @@ -1,33 +1,68 @@ package com.linkedin.hoptimator.k8s; +import java.sql.SQLException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.kubernetes.client.openapi.models.V1ObjectMeta; + +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.k8s.models.V1alpha1Pipeline; import com.linkedin.hoptimator.k8s.models.V1alpha1PipelineList; import com.linkedin.hoptimator.k8s.models.V1alpha1PipelineSpec; -import io.kubernetes.client.openapi.models.V1ObjectMeta; - -import java.sql.SQLException; -import java.util.List; -/** Deploys a Pipeline object. */ +/** + * Deploys a Pipeline object. Stamps {@code depends-on-*} labels and a {@code depends-on} + * collision-guard annotation describing which sources/sink the pipeline references, so + * {@link PipelineDependencyChecker} can look up dependents by label selector at delete time. + * + *

{@link K8sApi#update} merges labels additively, so stale {@code depends-on-*} labels from + * a previous version of the pipeline's SQL can linger. Correctness is preserved by the + * annotation, which is rewritten in full on every update: the checker rejects any label-only + * match whose annotation doesn't list the target identifier. In return we avoid the extra + * round trip that in-place label stripping would require. + */ class K8sPipelineDeployer extends K8sDeployer { private final String name; private final String yaml; private final String sql; + private final Collection sources; + private final Sink sink; K8sPipelineDeployer(String name, List specs, String sql, K8sContext context) { + this(name, specs, sql, Collections.emptyList(), null, context); + } + + K8sPipelineDeployer(String name, List specs, String sql, + Collection sources, Sink sink, K8sContext context) { super(context, K8sApiEndpoints.PIPELINES); this.name = name; this.yaml = String.join("\n---\n", specs); this.sql = sql; + this.sources = sources == null ? Collections.emptyList() : sources; + this.sink = sink; } @Override protected V1alpha1Pipeline toK8sObject() throws SQLException { + V1ObjectMeta meta = new V1ObjectMeta().name(name); + Map labels = PipelineDependencyLabels.labelsFor(sources, sink); + if (!labels.isEmpty()) { + meta.setLabels(new HashMap<>(labels)); + Map annotations = new HashMap<>(); + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY, + PipelineDependencyLabels.annotationFor(sources, sink)); + meta.setAnnotations(annotations); + } return new V1alpha1Pipeline().kind(K8sApiEndpoints.PIPELINES.kind()) .apiVersion(K8sApiEndpoints.PIPELINES.apiVersion()) - .metadata(new V1ObjectMeta().name(name)) + .metadata(meta) .spec(new V1alpha1PipelineSpec().sql(sql).yaml(yaml)); } } diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java new file mode 100644 index 00000000..7d3d955f --- /dev/null +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java @@ -0,0 +1,36 @@ +package com.linkedin.hoptimator.k8s; + +import java.sql.Connection; +import java.util.Collection; +import java.util.Collections; + +import com.linkedin.hoptimator.PendingDelete; +import com.linkedin.hoptimator.Source; +import com.linkedin.hoptimator.Validator; +import com.linkedin.hoptimator.ValidatorProvider; + + +/** + * Returns a Kubernetes-backed dependency-guard validator for any {@link Source} wrapped in a + * {@link PendingDelete} — i.e. when a DROP is issued. Keying off + * {@code PendingDelete} (not raw {@code Source}) makes the guard explicitly delete-time: + * other callers of {@code ValidationService.validateOrThrow(source, connection)} won't trigger + * a pre-delete lookup against K8s. + * + *

Registered via {@code META-INF/services/com.linkedin.hoptimator.ValidatorProvider}. + */ +public class K8sValidatorProvider implements ValidatorProvider { + + @Override + public Collection validators(T obj, Connection connection) { + if (obj instanceof PendingDelete) { + PendingDelete pd = (PendingDelete) obj; + Object target = pd.target(); + if (target instanceof Source) { + return Collections.singletonList( + new K8sPipelineDependencyValidator((Source) target, pd.selfOwnerUid())); + } + } + return Collections.emptyList(); + } +} diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sViewTable.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sViewTable.java index 401f6544..19a55611 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sViewTable.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sViewTable.java @@ -16,6 +16,7 @@ import org.apache.calcite.schema.impl.AbstractSchema; import org.apache.calcite.schema.impl.ViewTable; +import java.sql.Connection; import java.util.ArrayList; import java.util.List; import java.util.Objects; @@ -157,7 +158,7 @@ public Schema.TableType getJdbcTableType() { } @Override - public void validate(Validator.Issues issues) { + public void validate(Validator.Issues issues, Connection connection) { for (Row row : rows()) { Validator.Issues issues2 = issues.child(row.toString()); Validator.validateSubdomainName(row.NAME, issues2.child("NAME")); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java new file mode 100644 index 00000000..122f817d --- /dev/null +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java @@ -0,0 +1,111 @@ +package com.linkedin.hoptimator.k8s; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; + +import io.kubernetes.client.openapi.models.V1ObjectMeta; +import io.kubernetes.client.openapi.models.V1OwnerReference; + +import com.linkedin.hoptimator.k8s.models.V1alpha1Pipeline; +import com.linkedin.hoptimator.k8s.models.V1alpha1PipelineList; + + +/** + * Checks whether any Pipeline CRDs still depend on a resource a {@link com.linkedin.hoptimator.Deployer} + * is about to delete. + * + *

The lookup is a label-selector list against the Pipeline CRD group, so it is O(matches) on + * the wire — not a full scan. Each candidate is then cross-checked against the + * {@link PipelineDependencyLabels#ANNOTATION_KEY} annotation to rule out the (rare) case of a + * hash collision in the label slug. + * + *

Pipelines owned (directly) by {@code selfOwnerUid} are excluded from the blocker list: those + * pipelines will be cascade-deleted alongside the parent resource, so counting them as external + * dependents would make composite deletes (e.g. {@code LogicalTableDeployer.delete()}) impossible. + */ +public final class PipelineDependencyChecker { + + private PipelineDependencyChecker() { + } + + public static void assertNoExternalDependents(K8sContext context, String database, + List path, String selfOwnerUid) throws SQLException { + assertNoExternalDependents(new K8sApi<>(context, K8sApiEndpoints.PIPELINES), + database, path, selfOwnerUid); + } + + /** Variant that takes a pre-built {@link K8sApi} — used by tests to inject mocks. */ + static void assertNoExternalDependents(K8sApi api, + String database, List path, String selfOwnerUid) throws SQLException { + + String labelKey = PipelineDependencyLabels.labelKey(database, path); + String identifier = PipelineDependencyLabels.identifier(database, path); + + Collection matches = api.select(labelKey); + + List blockers = new ArrayList<>(); + for (V1alpha1Pipeline p : matches) { + if (isSelfOwned(p, selfOwnerUid)) { + continue; + } + if (!annotationConfirms(p, identifier)) { + // Label matched but annotation doesn't — this is a slug collision, skip it. + continue; + } + blockers.add(describeBlocker(p)); + } + + if (!blockers.isEmpty()) { + throw new SQLException(String.format( + "Cannot delete %s — %d active pipeline(s) depend on it: %s", + identifier, blockers.size(), String.join(", ", blockers))); + } + } + + private static boolean isSelfOwned(V1alpha1Pipeline pipeline, String selfOwnerUid) { + if (selfOwnerUid == null) { + return false; + } + V1ObjectMeta meta = pipeline.getMetadata(); + if (meta == null || meta.getOwnerReferences() == null) { + return false; + } + for (V1OwnerReference owner : meta.getOwnerReferences()) { + if (selfOwnerUid.equals(owner.getUid())) { + return true; + } + } + return false; + } + + private static boolean annotationConfirms(V1alpha1Pipeline pipeline, String identifier) { + V1ObjectMeta meta = pipeline.getMetadata(); + if (meta == null || meta.getAnnotations() == null) { + return true; // pre-labeling pipeline — conservatively trust the label match + } + String annotation = meta.getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY); + if (annotation == null) { + return true; // same — no annotation to cross-check against + } + Set listed = PipelineDependencyLabels.parseAnnotation(annotation); + return listed.contains(identifier); + } + + /** + * Builds a human-readable blocker description: the pipeline name, plus (when present) the top + * ownerReference's {@code kind/name} so the user knows which higher-level resource owns it. + */ + private static String describeBlocker(V1alpha1Pipeline pipeline) { + V1ObjectMeta meta = pipeline.getMetadata(); + String name = meta == null ? "" : meta.getName(); + String ownerSuffix = ""; + if (meta != null && meta.getOwnerReferences() != null && !meta.getOwnerReferences().isEmpty()) { + V1OwnerReference owner = meta.getOwnerReferences().get(0); + ownerSuffix = " (owned by " + owner.getKind() + "/" + owner.getName() + ")"; + } + return name + ownerSuffix; + } +} diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java new file mode 100644 index 00000000..6d4004bd --- /dev/null +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java @@ -0,0 +1,144 @@ +package com.linkedin.hoptimator.k8s; + +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; + + +/** + * Computes the labels and annotation that encode a Pipeline CRD's dependency edges. + * + *

Every source and sink a pipeline references is recorded as a label: + * {@code hoptimator.linkedin.com/depends-on-: "/"} where + * {@code } is a deterministic hash derived from {@code database + "/" + pathString}. + * The hash keeps label keys within Kubernetes's 63-character name limit for arbitrary paths, + * and lets {@code K8sApi.select} filter pipelines by dependency on the server. + * + *

A collision-guard annotation ({@code ANNOTATION_KEY}) lists all logical identifiers verbatim, + * so the delete-time check can distinguish a real dependency match from a rare hash collision. + */ +public final class PipelineDependencyLabels { + + static final String LABEL_PREFIX = "hoptimator.linkedin.com/depends-on-"; + public static final String ANNOTATION_KEY = "hoptimator.linkedin.com/depends-on"; + + private static final int SLUG_LENGTH = 16; // 64 bits of SHA-256 → ~1 in 1.8e19 collisions + private static final int MAX_LABEL_VALUE = 63; + + private PipelineDependencyLabels() { + } + + /** + * Canonical logical identifier for a resource: {@code _}. + * + *

The separator is {@code _} (not {@code /}) so the result is also a valid Kubernetes + * label value out of the box — K8s allows {@code [A-Za-z0-9_.-]} but not {@code /}. + */ + public static String identifier(String database, Iterable path) { + return database + "_" + String.join(".", path); + } + + /** Hex slug derived from the full identifier; same identifier always produces the same slug. */ + public static String slug(String database, Iterable path) { + byte[] digest = sha256(identifier(database, path).getBytes(StandardCharsets.UTF_8)); + StringBuilder sb = new StringBuilder(SLUG_LENGTH); + for (int i = 0; i < SLUG_LENGTH / 2; i++) { + sb.append(String.format("%02x", digest[i])); + } + return sb.toString(); + } + + /** Label key a Pipeline carries if it depends on the given resource. */ + public static String labelKey(String database, Iterable path) { + return LABEL_PREFIX + slug(database, path); + } + + /** + * Labels to stamp on a Pipeline CRD — one entry per source and the sink. Both edges + * matter to the guard: dropping a source orphans pipelines that read from it; dropping a sink + * orphans pipelines that write to it. The partial-view scenario where two pipelines share a + * sink (e.g. {@code X} and {@code X$piece}) is unaffected — DROP MV routes through + * {@code K8sViewDeployer}, which deliberately does not implement {@code DependencyGuarded} + * (DROP MV is metadata-only and does not destroy the underlying physical sink). + * + *

Keys are the same as {@link #labelKey}. Values are the readable identifier, truncated + * to 63 chars if necessary (the annotation preserves the untruncated form). + */ + public static Map labelsFor(Collection sources, Sink sink) { + Map labels = new LinkedHashMap<>(); + for (Source src : sources) { + labels.put(labelKey(src.database(), src.path()), truncate(identifier(src.database(), src.path()))); + } + if (sink != null) { + labels.put(labelKey(sink.database(), sink.path()), truncate(identifier(sink.database(), sink.path()))); + } + return labels; + } + + /** + * Collision-guard annotation value — comma-separated list of full source and sink identifiers, + * deduplicated. The delete-time check cross-references this annotation after the label + * selector narrows the candidate set. + */ + public static String annotationFor(Collection sources, Sink sink) { + Set ids = new LinkedHashSet<>(); + for (Source src : sources) { + ids.add(identifier(src.database(), src.path())); + } + if (sink != null) { + ids.add(identifier(sink.database(), sink.path())); + } + return String.join(",", ids); + } + + /** + * Removes any {@code depends-on-*} entries from an existing label map so that an update can + * restamp the current dependency set without carrying stale labels from an earlier version of + * the pipeline. {@link K8sApi#update} is additive for labels; callers must strip first. + */ + public static Map stripDependencyLabels(Map labels) { + if (labels == null) { + return new LinkedHashMap<>(); + } + return labels.entrySet().stream() + .filter(e -> !e.getKey().startsWith(LABEL_PREFIX)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, + (a, b) -> a, LinkedHashMap::new)); + } + + /** Parses the collision-guard annotation back into the set of identifiers it encoded. */ + public static Set parseAnnotation(String annotation) { + Set out = new LinkedHashSet<>(); + if (annotation == null || annotation.isEmpty()) { + return out; + } + for (String id : annotation.split(",")) { + String trimmed = id.trim(); + if (!trimmed.isEmpty()) { + out.add(trimmed); + } + } + return out; + } + + private static String truncate(String value) { + return value.length() <= MAX_LABEL_VALUE ? value : value.substring(0, MAX_LABEL_VALUE); + } + + private static byte[] sha256(byte[] input) { + try { + return MessageDigest.getInstance("SHA-256").digest(input); + } catch (NoSuchAlgorithmException e) { + throw new IllegalStateException("SHA-256 unavailable", e); + } + } +} diff --git a/hoptimator-k8s/src/main/resources/META-INF/services/com.linkedin.hoptimator.ValidatorProvider b/hoptimator-k8s/src/main/resources/META-INF/services/com.linkedin.hoptimator.ValidatorProvider new file mode 100644 index 00000000..5fda4b5c --- /dev/null +++ b/hoptimator-k8s/src/main/resources/META-INF/services/com.linkedin.hoptimator.ValidatorProvider @@ -0,0 +1 @@ +com.linkedin.hoptimator.k8s.K8sValidatorProvider diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployerTest.java index 2acb55e0..5b987339 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployerTest.java @@ -8,7 +8,6 @@ import com.linkedin.hoptimator.SqlDialect; import com.linkedin.hoptimator.ThrowingFunction; import com.linkedin.hoptimator.util.DeploymentService; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.kubernetes.client.openapi.models.V1OwnerReference; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -19,6 +18,7 @@ import java.sql.SQLException; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; @@ -33,8 +33,6 @@ @ExtendWith(MockitoExtension.class) -@SuppressFBWarnings(value = {"OBL_UNSATISFIED_OBLIGATION"}, - justification = "Mock objects created in stubbing setup don't need resource management") class K8sMaterializedViewDeployerTest { @Mock @@ -75,7 +73,7 @@ K8sViewDeployer createViewDeployer(MaterializedView v, K8sContext ctx) { @Override K8sPipelineBundle createPipelineBundle(String name, List pipelineSpecs, String sql, - K8sContext viewContext) { + Collection sources, Sink sink, K8sContext viewContext) { return capturedBundle; } }; diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineBundleTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineBundleTest.java index 396271fe..dc84bb62 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineBundleTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineBundleTest.java @@ -9,9 +9,13 @@ import java.sql.SQLException; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.ArgumentMatchers.any; @@ -42,9 +46,10 @@ void setUp() { private K8sPipelineBundle makeBundleWithMocks(String name, List pipelineSpecs) { K8sPipelineDeployer capturedPipelineDeployer = pipelineDeployer; K8sYamlDeployerImpl capturedYamlDeployer = yamlDeployer; - return new K8sPipelineBundle(name, pipelineSpecs, "INSERT INTO sink SELECT * FROM source", context) { + return new K8sPipelineBundle(name, pipelineSpecs, "INSERT INTO sink SELECT * FROM source", Collections.emptyList(), null, context) { @Override - K8sPipelineDeployer createPipelineDeployer(String n, List specs, String sql, K8sContext ctx) { + K8sPipelineDeployer createPipelineDeployer(String n, List specs, String sql, + Collection sources, Sink sink, K8sContext ctx) { return capturedPipelineDeployer; } diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java index c7ef7469..3af0f1dd 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java @@ -1,18 +1,32 @@ package com.linkedin.hoptimator.k8s; +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.k8s.models.V1alpha1Pipeline; import org.junit.jupiter.api.Test; import java.sql.SQLException; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; class K8sPipelineDeployerTest { + private static Source src(String db, String path) { + return new Source(db, Collections.singletonList(path), Collections.emptyMap()); + } + + private static Sink sink(String db, String path) { + return new Sink(db, Collections.singletonList(path), Collections.emptyMap()); + } + @Test void toK8sObjectSetsPipelineFields() throws SQLException { K8sPipelineDeployer deployer = new K8sPipelineDeployer( @@ -36,4 +50,52 @@ void toK8sObjectWithSingleSpec() throws SQLException { assertEquals("only-spec", pipeline.getSpec().getYaml()); } + + @Test + void legacyConstructorOmitsDependencyLabelsAndAnnotation() throws SQLException { + K8sPipelineDeployer deployer = new K8sPipelineDeployer( + "legacy", List.of("spec"), "SELECT 1", null); + + V1alpha1Pipeline pipeline = deployer.toK8sObject(); + + assertNull(pipeline.getMetadata().getLabels(), + "legacy constructor must not stamp any labels"); + assertNull(pipeline.getMetadata().getAnnotations(), + "legacy constructor must not stamp any annotations"); + } + + @Test + void stampsDependencyLabelsForSourcesAndSink() throws SQLException { + K8sPipelineDeployer deployer = new K8sPipelineDeployer( + "p1", List.of("spec"), "SELECT 1", + Arrays.asList(src("kafka1", "topic-a"), src("kafka2", "topic-b")), + sink("mysql", "outbox"), null); + + V1alpha1Pipeline pipeline = deployer.toK8sObject(); + Map labels = pipeline.getMetadata().getLabels(); + + assertEquals(3, labels.size(), "should have one label per source + one for the sink"); + assertTrue(labels.containsKey( + PipelineDependencyLabels.labelKey("kafka1", Collections.singletonList("topic-a")))); + assertTrue(labels.containsKey( + PipelineDependencyLabels.labelKey("kafka2", Collections.singletonList("topic-b")))); + assertTrue(labels.containsKey( + PipelineDependencyLabels.labelKey("mysql", Collections.singletonList("outbox")))); + } + + @Test + void stampsCollisionGuardAnnotation() throws SQLException { + K8sPipelineDeployer deployer = new K8sPipelineDeployer( + "p1", List.of("spec"), "SELECT 1", + Collections.singletonList(src("kafka", "topic")), + sink("mysql", "outbox"), null); + + V1alpha1Pipeline pipeline = deployer.toK8sObject(); + Map annotations = pipeline.getMetadata().getAnnotations(); + + String annotation = annotations.get(PipelineDependencyLabels.ANNOTATION_KEY); + assertNotNull(annotation); + assertTrue(annotation.contains("kafka_topic")); + assertTrue(annotation.contains("mysql_outbox")); + } } diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sViewTableTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sViewTableTest.java index a7a50130..8e817d15 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sViewTableTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sViewTableTest.java @@ -198,7 +198,7 @@ void validateWithValidRows() { stubRows(); Validator.Issues issues = new Validator.Issues("test"); - tableWithApi.validate(issues); + tableWithApi.validate(issues, null); assertNotNull(issues); } @@ -213,7 +213,7 @@ void validateDetectsDuplicateNames() { stubRows(); Validator.Issues issues = new Validator.Issues("test"); - tableWithApi.validate(issues); + tableWithApi.validate(issues, null); assertNotNull(issues); } @@ -402,7 +402,7 @@ void validateWithDuplicateNameRecordsError() { stubRows(); Validator.Issues issues = new Validator.Issues("test"); - tableWithApi.validate(issues); + tableWithApi.validate(issues, null); assertFalse(issues.valid(), "Duplicate view name must record an error, making issues invalid"); @@ -419,7 +419,7 @@ void validateWithUniqueValidNamesIsValid() { stubRows(); Validator.Issues issues = new Validator.Issues("test"); - tableWithApi.validate(issues); + tableWithApi.validate(issues, null); assertTrue(issues.valid(), "unique valid names must not produce errors"); } diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java new file mode 100644 index 00000000..ab06c6e4 --- /dev/null +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java @@ -0,0 +1,137 @@ +package com.linkedin.hoptimator.k8s; + +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.kubernetes.client.openapi.models.V1ObjectMeta; +import io.kubernetes.client.openapi.models.V1OwnerReference; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import com.linkedin.hoptimator.k8s.models.V1alpha1Pipeline; +import com.linkedin.hoptimator.k8s.models.V1alpha1PipelineList; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.when; + + +@ExtendWith(MockitoExtension.class) +class PipelineDependencyCheckerTest { + + @Mock + private K8sApi api; + + private static final String DB = "kafka1"; + private static final List PATH = Collections.singletonList("my-topic"); + private static final String IDENTIFIER = "kafka1_my-topic"; + + private static V1alpha1Pipeline pipeline(String name, String ownerUid, String annotationValue) { + V1ObjectMeta meta = new V1ObjectMeta().name(name); + if (ownerUid != null) { + meta.addOwnerReferencesItem(new V1OwnerReference().kind("View").name("owner").uid(ownerUid)); + } + if (annotationValue != null) { + Map annotations = new HashMap<>(); + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY, annotationValue); + meta.setAnnotations(annotations); + } + return new V1alpha1Pipeline().metadata(meta); + } + + @Test + void passesWhenNoPipelinesMatch() throws SQLException { + when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Collections.emptyList()); + + assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + } + + @Test + void blocksOnExternalPipeline() throws SQLException { + when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + .thenReturn(Collections.singletonList(pipeline("ext-pipe", "other-uid", IDENTIFIER))); + + SQLException ex = assertThrows(SQLException.class, + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + assertTrue(ex.getMessage().contains("ext-pipe")); + assertTrue(ex.getMessage().contains(IDENTIFIER)); + } + + @Test + void skipsSelfOwnedPipeline() throws SQLException { + when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + .thenReturn(Collections.singletonList(pipeline("owned-pipe", "self-uid", IDENTIFIER))); + + assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( + api, DB, PATH, "self-uid")); + } + + @Test + void blocksOnExternalWhenSomeAreSelfOwned() throws SQLException { + when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( + pipeline("owned-pipe", "self-uid", IDENTIFIER), + pipeline("external-pipe", "other-uid", IDENTIFIER))); + + SQLException ex = assertThrows(SQLException.class, + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, "self-uid")); + assertTrue(ex.getMessage().contains("external-pipe")); + assertTrue(!ex.getMessage().contains("owned-pipe"), "self-owned pipeline must not be listed"); + } + + @Test + void rejectsSlugCollisionViaAnnotation() throws SQLException { + // Pipeline labels collide on the slug (which is what api.select matched on) but the + // annotation reveals the actual identifier is different — so this should NOT block. + when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + .thenReturn(Collections.singletonList(pipeline("colliding-pipe", "other-uid", + "some-other-database/some-other-path"))); + + assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + } + + @Test + void treatsMissingAnnotationAsTrusted() throws SQLException { + // A pipeline with the matching label but no depends-on annotation (pre-labeling migration + // case, or future code path that didn't write the annotation) is still treated as a blocker. + when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + .thenReturn(Collections.singletonList(pipeline("legacy-pipe", "other-uid", null))); + + SQLException ex = assertThrows(SQLException.class, + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + assertTrue(ex.getMessage().contains("legacy-pipe")); + } + + @Test + void errorMessageIncludesOwnerKindAndName() throws SQLException { + when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + .thenReturn(Collections.singletonList(pipeline("ext-pipe", "other-uid", IDENTIFIER))); + + SQLException ex = assertThrows(SQLException.class, + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + assertTrue(ex.getMessage().contains("View/owner"), + "error should name the owning View so the user knows what to unhook: " + ex.getMessage()); + } + + @Test + void errorMessageListsAllBlockers() throws SQLException { + when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( + pipeline("p1", "uid1", IDENTIFIER), + pipeline("p2", "uid2", IDENTIFIER), + pipeline("p3", "uid3", IDENTIFIER))); + + SQLException ex = assertThrows(SQLException.class, + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + assertTrue(ex.getMessage().contains("p1")); + assertTrue(ex.getMessage().contains("p2")); + assertTrue(ex.getMessage().contains("p3")); + assertTrue(ex.getMessage().contains("3 active pipeline")); + } +} diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java new file mode 100644 index 00000000..68482c5e --- /dev/null +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java @@ -0,0 +1,184 @@ +package com.linkedin.hoptimator.k8s; + +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.junit.jupiter.api.Test; + +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +class PipelineDependencyLabelsTest { + + private static Source src(String db, String... path) { + return new Source(db, Arrays.asList(path), Collections.emptyMap()); + } + + private static Sink sink(String db, String... path) { + return new Sink(db, Arrays.asList(path), Collections.emptyMap()); + } + + @Test + void identifierJoinsDatabaseAndPath() { + // Separator is "_" so the identifier is also a valid K8s label value out of the box. + assertEquals("mydb_a.b.c", PipelineDependencyLabels.identifier("mydb", Arrays.asList("a", "b", "c"))); + } + + @Test + void slugIsDeterministic() { + String s1 = PipelineDependencyLabels.slug("db", Arrays.asList("foo", "bar")); + String s2 = PipelineDependencyLabels.slug("db", Arrays.asList("foo", "bar")); + assertEquals(s1, s2); + } + + @Test + void slugVariesByDatabase() { + String a = PipelineDependencyLabels.slug("db1", Collections.singletonList("t")); + String b = PipelineDependencyLabels.slug("db2", Collections.singletonList("t")); + assertNotEquals(a, b); + } + + @Test + void slugVariesByPath() { + String a = PipelineDependencyLabels.slug("db", Arrays.asList("schema", "t")); + String b = PipelineDependencyLabels.slug("db", Arrays.asList("schema", "u")); + assertNotEquals(a, b); + } + + @Test + void labelKeyFitsKubernetesNameLimit() { + // Long path stressing the slug — name portion (after the /) must be ≤ 63 chars. + String key = PipelineDependencyLabels.labelKey( + "a-really-long-database-name", + Arrays.asList("catalog", "schema", "a_very_long_table_name_that_exceeds_sixty_three_chars")); + String namePortion = key.substring(key.indexOf('/') + 1); + assertTrue(namePortion.length() <= 63, "name portion must be ≤63 chars, got " + namePortion.length()); + assertTrue(namePortion.matches("[a-z0-9]([-a-z0-9_.]*[a-z0-9])?"), + "name portion must match K8s label-name regex, got: " + namePortion); + } + + @Test + void labelsForIncludesSourcesAndSink() { + // Both edges matter: dropping a source orphans readers; dropping a sink orphans writers. + Source s1 = src("kafka1", "events"); + Source s2 = src("venice1", "store"); + Sink sink = sink("mysql1", "outbox"); + Map labels = PipelineDependencyLabels.labelsFor(Arrays.asList(s1, s2), sink); + + assertEquals(3, labels.size()); + assertTrue(labels.containsKey(PipelineDependencyLabels.labelKey("kafka1", Collections.singletonList("events")))); + assertTrue(labels.containsKey(PipelineDependencyLabels.labelKey("venice1", Collections.singletonList("store")))); + assertTrue(labels.containsKey(PipelineDependencyLabels.labelKey("mysql1", Collections.singletonList("outbox")))); + } + + @Test + void labelsForHandlesNullSink() { + Map labels = PipelineDependencyLabels.labelsFor( + Collections.singletonList(src("db", "t")), null); + assertEquals(1, labels.size()); + } + + @Test + void labelsForCollapsesSelfLoopIntoOneEntry() { + // Self-loop pipeline: source and sink share a slug, so the map collapses to one entry + // rather than producing duplicate keys. + Source s = src("db", "t"); + Sink k = sink("db", "t"); + Map labels = PipelineDependencyLabels.labelsFor(Collections.singletonList(s), k); + assertEquals(1, labels.size()); + } + + @Test + void labelValueTruncatedAtSixtyThreeChars() { + String longPath = "this_is_a_really_long_table_name_that_exceeds_sixty_three_chars_by_a_lot"; + Map labels = PipelineDependencyLabels.labelsFor( + Collections.singletonList(src("db", longPath)), null); + String value = labels.values().iterator().next(); + assertTrue(value.length() <= 63); + } + + @Test + void labelValueIsKubernetesLabelValueCompliant() { + // K8s label values must match (([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])? + // — the identifier separator is "_" precisely so this holds out of the box for typical + // (database, path) shapes seen in production. + Map labels = PipelineDependencyLabels.labelsFor( + Collections.singletonList(src("ads-database", "ADS", "PAGE_VIEWS")), null); + String value = labels.values().iterator().next(); + + assertTrue(value.length() <= 63); + assertTrue(value.matches("(([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])?"), + "value must satisfy K8s label-value regex, got: " + value); + assertFalse(value.contains("/"), "no '/' separator should leak into the label value"); + } + + @Test + void annotationForListsAllIdentifiers() { + // Sources and the sink — both edges are recorded so the delete-time check can disambiguate + // a real dependency from a hash collision regardless of which side matched. + String annotation = PipelineDependencyLabels.annotationFor( + Arrays.asList(src("kafka", "a"), src("venice", "b")), + sink("mysql", "c")); + assertTrue(annotation.contains("kafka_a")); + assertTrue(annotation.contains("venice_b")); + assertTrue(annotation.contains("mysql_c")); + } + + @Test + void annotationForDeduplicatesAndOmitsNullSink() { + String annotation = PipelineDependencyLabels.annotationFor( + Arrays.asList(src("db", "t"), src("db", "t")), null); + assertEquals("db_t", annotation); + } + + @Test + void parseAnnotationRoundtrip() { + String annotation = PipelineDependencyLabels.annotationFor( + Arrays.asList(src("a", "1"), src("b", "2")), sink("c", "3")); + Set parsed = PipelineDependencyLabels.parseAnnotation(annotation); + assertEquals(3, parsed.size()); + assertTrue(parsed.contains("a_1")); + assertTrue(parsed.contains("b_2")); + assertTrue(parsed.contains("c_3")); + } + + @Test + void parseAnnotationHandlesNullAndEmpty() { + assertTrue(PipelineDependencyLabels.parseAnnotation(null).isEmpty()); + assertTrue(PipelineDependencyLabels.parseAnnotation("").isEmpty()); + assertTrue(PipelineDependencyLabels.parseAnnotation(" , ").isEmpty()); + } + + @Test + void stripDependencyLabelsRemovesOnlyPrefixedEntries() { + Map existing = new LinkedHashMap<>(); + existing.put("app", "hoptimator"); // unrelated label, keep + existing.put(PipelineDependencyLabels.labelKey("db", List.of("t")), "db/t"); // strip + existing.put("pipeline", "my-pipeline"); // keep + + Map stripped = PipelineDependencyLabels.stripDependencyLabels(existing); + + assertEquals(2, stripped.size()); + assertTrue(stripped.containsKey("app")); + assertTrue(stripped.containsKey("pipeline")); + assertFalse(stripped.containsKey(PipelineDependencyLabels.labelKey("db", List.of("t")))); + } + + @Test + void stripDependencyLabelsHandlesNull() { + Map result = PipelineDependencyLabels.stripDependencyLabels(null); + assertNotNull(result, "null input must return a non-null empty map, not propagate the null"); + assertTrue(result.isEmpty()); + } +} diff --git a/hoptimator-kafka/src/main/java/com/linkedin/hoptimator/kafka/KafkaDeployer.java b/hoptimator-kafka/src/main/java/com/linkedin/hoptimator/kafka/KafkaDeployer.java index 6b9e8fc5..3a269d1e 100644 --- a/hoptimator-kafka/src/main/java/com/linkedin/hoptimator/kafka/KafkaDeployer.java +++ b/hoptimator-kafka/src/main/java/com/linkedin/hoptimator/kafka/KafkaDeployer.java @@ -19,6 +19,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.sql.Connection; import java.sql.SQLException; import java.time.Duration; import java.util.Collections; @@ -53,7 +54,7 @@ public KafkaDeployer(Source source, Properties properties) { } @Override - public void validate(Validator.Issues issues) { + public void validate(Validator.Issues issues, Connection connection) { String topicName = source.table(); // null default = option was not specified by user, skip validation for that option Integer partitions = DeployerUtils.parseIntOption(source.options(), "partitions", null); diff --git a/hoptimator-kafka/src/test/java/com/linkedin/hoptimator/kafka/KafkaDeployerTest.java b/hoptimator-kafka/src/test/java/com/linkedin/hoptimator/kafka/KafkaDeployerTest.java index 8758bcbf..42848533 100644 --- a/hoptimator-kafka/src/test/java/com/linkedin/hoptimator/kafka/KafkaDeployerTest.java +++ b/hoptimator-kafka/src/test/java/com/linkedin/hoptimator/kafka/KafkaDeployerTest.java @@ -568,7 +568,7 @@ private TopicDescription mockTopicWithPartitions(int numPartitions) { private Validator.Issues collectIssues(KafkaDeployer deployer) { Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); return issues; } diff --git a/hoptimator-mysql/src/main/java/com/linkedin/hoptimator/mysql/MySqlDeployer.java b/hoptimator-mysql/src/main/java/com/linkedin/hoptimator/mysql/MySqlDeployer.java index c2d996d4..227600bb 100644 --- a/hoptimator-mysql/src/main/java/com/linkedin/hoptimator/mysql/MySqlDeployer.java +++ b/hoptimator-mysql/src/main/java/com/linkedin/hoptimator/mysql/MySqlDeployer.java @@ -31,8 +31,8 @@ /** * Deployer for MySQL tables. Creates tables in the synchronous DDL hot path. * - *

Implements {@link Validated} to pre-check table constraints - * before any deployment side effects. + *

Implements {@link Validated} to pre-check table constraints before any deployment side + * effects. */ public class MySqlDeployer implements Deployer, Validated { @@ -99,7 +99,7 @@ private String toMySqlType(RelDataTypeField field) { } @Override - public void validate(Validator.Issues issues) { + public void validate(Validator.Issues issues, Connection connection) { String tableName = source.table(); String database = source.schema(); @@ -170,11 +170,11 @@ public void validate(Validator.Issues issues) { if (columnName.equals(keyField)) { String newType = toMySqlType(field); String existingType = existingCol.type; - + // Normalize types for comparison (remove size for basic comparison) String normalizedNew = newType.replaceAll("\\(.*?\\)", ""); String normalizedExisting = existingType.replaceAll("\\(.*?\\)", ""); - + if (!normalizedNew.equalsIgnoreCase(normalizedExisting)) { issues.error("Cannot modify KEY field type for table " + tableName + ". KEY field '" + keyField + "' has existing type " + existingType diff --git a/hoptimator-mysql/src/test/java/com/linkedin/hoptimator/mysql/MySqlDeployerTest.java b/hoptimator-mysql/src/test/java/com/linkedin/hoptimator/mysql/MySqlDeployerTest.java index 660cabac..08f352cf 100644 --- a/hoptimator-mysql/src/test/java/com/linkedin/hoptimator/mysql/MySqlDeployerTest.java +++ b/hoptimator-mysql/src/test/java/com/linkedin/hoptimator/mysql/MySqlDeployerTest.java @@ -1,10 +1,10 @@ package com.linkedin.hoptimator.mysql; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.Validator; import com.linkedin.hoptimator.jdbc.HoptimatorConnection; import com.linkedin.hoptimator.jdbc.HoptimatorDriver; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeSystem; @@ -116,7 +116,7 @@ private MySqlDeployer createDeployer(Source source) { private Validator.Issues collectIssues(MySqlDeployer deployer) { Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); return issues; } @@ -406,7 +406,7 @@ void testValidateFailsWithNullDatabase() { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Database & table names are required")); @@ -418,7 +418,7 @@ void testValidateFailsWithInvalidDatabaseName() { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Invalid database name")); @@ -430,7 +430,7 @@ void testValidateFailsWithInvalidTableName() { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Invalid table name")); @@ -442,7 +442,7 @@ void testValidateFailsWithEmptyDatabaseName() { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Invalid database name")); @@ -469,7 +469,7 @@ void testValidateFailsNoKeyFields() throws SQLException { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("No KEY_ fields found")); @@ -504,7 +504,7 @@ void testValidateFailsWhenPrimaryKeysChange() throws SQLException { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Cannot modify KEY fields")); @@ -531,7 +531,7 @@ void testValidateFailsWithInvalidColumnName() throws SQLException { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Invalid column name")); @@ -547,7 +547,7 @@ void testValidateFailsWhenRowTypeThrowsException() throws SQLException { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Failed to get schema for table")); @@ -568,7 +568,7 @@ void testValidatePassesWithMaxLength64Identifier() throws SQLException { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertTrue(issues.valid(), "Expected 64-char identifier to be valid"); } @@ -581,7 +581,7 @@ void testValidateFailsWithTooLongIdentifier() { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid(), "Expected 65-char identifier to be invalid"); assertTrue(issues.toString().contains("Invalid table name"), @@ -601,7 +601,7 @@ void testValidatePassesWhenAllConditionsGood() throws SQLException { MySqlDeployer deployer = new MySqlDeployer(source, PROPERTIES, mockHoptimatorConnection); Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertTrue(issues.valid(), "Expected no errors for valid new table, got: " + issues); } @@ -1311,4 +1311,5 @@ void testEnsureDatabaseExistsSqlContainsCreateDatabase() throws Exception { assertTrue(createDbSql.contains("`test_db`"), "Expected backtick-escaped db name in CREATE DATABASE SQL, got: " + createDbSql); } + } diff --git a/hoptimator-util/src/test/java/com/linkedin/hoptimator/util/DeploymentServiceTest.java b/hoptimator-util/src/test/java/com/linkedin/hoptimator/util/DeploymentServiceTest.java index 0220f98b..554b71df 100644 --- a/hoptimator-util/src/test/java/com/linkedin/hoptimator/util/DeploymentServiceTest.java +++ b/hoptimator-util/src/test/java/com/linkedin/hoptimator/util/DeploymentServiceTest.java @@ -23,6 +23,7 @@ import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; + import java.net.URLEncoder; import java.nio.charset.StandardCharsets; import java.sql.Connection; @@ -181,7 +182,6 @@ void parseHintsMixedEncodedAndNonEncoded() { assertEquals("test", mixed.get("another")); } - @Test void testCreateCallsCreateOnAllDeployers() throws SQLException { List deployers = Arrays.asList(mockDeployer1, mockDeployer2); diff --git a/hoptimator-venice/src/main/java/com/linkedin/hoptimator/venice/VeniceDeployer.java b/hoptimator-venice/src/main/java/com/linkedin/hoptimator/venice/VeniceDeployer.java index 28f34361..ee61a1b3 100644 --- a/hoptimator-venice/src/main/java/com/linkedin/hoptimator/venice/VeniceDeployer.java +++ b/hoptimator-venice/src/main/java/com/linkedin/hoptimator/venice/VeniceDeployer.java @@ -25,6 +25,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.sql.Connection; import java.sql.SQLException; import java.sql.SQLNonTransientException; import java.util.Collections; @@ -55,7 +56,7 @@ public VeniceDeployer(Source source, Properties properties, HoptimatorConnection } @Override - public void validate(Validator.Issues issues) { + public void validate(Validator.Issues issues, Connection connection) { String storeName = source.table(); // Validate Venice configuration diff --git a/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/VeniceDeployerTest.java b/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/VeniceDeployerTest.java index b364cc29..9a16cea9 100644 --- a/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/VeniceDeployerTest.java +++ b/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/VeniceDeployerTest.java @@ -10,7 +10,6 @@ import com.linkedin.venice.controllerapi.SchemaResponse; import com.linkedin.venice.controllerapi.StoreResponse; import com.linkedin.venice.meta.StoreInfo; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.avro.Schema; import org.apache.calcite.util.Pair; import org.junit.jupiter.api.BeforeEach; @@ -40,8 +39,6 @@ * Tests for VeniceDeployer using mocks. */ @ExtendWith(MockitoExtension.class) -@SuppressFBWarnings(value = {"OBL_UNSATISFIED_OBLIGATION", "ODR_OPEN_DATABASE_RESOURCE"}, - justification = "Mock objects created in stubbing setup don't need resource management") class VeniceDeployerTest { private static final String TEST_STORE = "test_store"; @@ -315,7 +312,7 @@ protected Pair getKeyPayloadSchema() throws SQLException { }; Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertTrue(issues.valid(), "Expected no validation errors for new store. Issues: " + issues); } @@ -348,7 +345,7 @@ protected Pair getKeyPayloadSchema() throws SQLException { }; Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertTrue(issues.valid(), "Expected no validation errors when key schema unchanged. Issues: " + issues); } @@ -381,7 +378,7 @@ protected Pair getKeyPayloadSchema() throws SQLException { }; Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid(), "Expected validation error for key schema change"); assertTrue(issues.toString().contains("Key schema evolution is not supported"), @@ -400,7 +397,7 @@ protected Pair getKeyPayloadSchema() { }; Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Failed to generate key schema")); @@ -418,7 +415,7 @@ protected Pair getKeyPayloadSchema() { }; Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Failed to generate value schema")); @@ -436,10 +433,11 @@ protected Pair getKeyPayloadSchema() { }; Validator.Issues issues = new Validator.Issues("test"); - deployer.validate(issues); + deployer.validate(issues, null); assertFalse(issues.valid()); assertTrue(issues.toString().contains("Failed to generate key schema")); assertTrue(issues.toString().contains("Failed to generate value schema")); } + } From f78802c5132a6ffc2cf2aaa5696ef800b070e26d Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Thu, 30 Apr 2026 21:37:17 -0400 Subject: [PATCH 02/15] feat: support DROP TABLE for logical tables MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit LogicalTableDeployer.delete() previously threw SQLFeatureNotSupported. Now implemented end-to-end as a per-tier sequence that mirrors what running DROP TABLE on each tier independently would do, plus the LogicalTable CRD removal at the top. Flow: 1. Per-tier pre-flight via the validator framework: ValidationService.validateOrThrow(new PendingDelete<>(tierSource, logicalTableUid), connection) — refuses the drop if any active external pipeline still references a tier resource. The selfOwnerUid is the LogicalTable CRD's UID so the implicit inter-tier pipelines (owned by the CRD, cascade-deleted with it) don't self-block. 2. Delete the LogicalTable CRD. K8s owner-ref cascade removes its owned Pipeline and TableTrigger CRDs. 3. Best-effort physical cleanup of each tier resource (Kafka topic, Venice store, ...). A failed tier delete logs a warning but does not abort — a stranded tier is recoverable; aborting mid-DROP isn't. 4. Per-tier schema cleanup: deregister the TemporaryTable entry in each tier schema only when its physical delete succeeded. Tests: - LogicalTableDeployerTest deleteRemovesCrdAndCleansUpTierResources, deletePropagatesCrdDeletionFailure, deleteSwallowsTierCleanupFailures. - logical-ddl.id integration test: DROP TABLE LOGICAL.testevent now succeeds and cascades the implicit nearline-to-online pipeline. - logical-offline-ddl.id companion update. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../logical/LogicalTableDeployer.java | 99 ++++++-- .../logical/LogicalTableDeployerTest.java | 236 +++++++++++++++--- .../src/test/resources/logical-ddl.id | 65 ++++- .../src/test/resources/logical-offline-ddl.id | 5 + 4 files changed, 349 insertions(+), 56 deletions(-) diff --git a/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java b/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java index 59258791..b2daf305 100644 --- a/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java +++ b/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java @@ -1,7 +1,7 @@ package com.linkedin.hoptimator.logical; +import java.sql.Connection; import java.sql.SQLException; -import java.sql.SQLFeatureNotSupportedException; import java.sql.SQLNonTransientException; import java.util.ArrayList; import java.util.Collection; @@ -16,6 +16,8 @@ import com.linkedin.hoptimator.k8s.models.V1alpha1DatabaseSpec; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplate; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplateList; +import com.linkedin.hoptimator.k8s.models.V1alpha1LogicalTable; +import com.linkedin.hoptimator.k8s.models.V1alpha1LogicalTableList; import com.linkedin.hoptimator.k8s.models.V1alpha1TableTrigger; import com.linkedin.hoptimator.k8s.models.V1alpha1TableTriggerList; import com.linkedin.hoptimator.util.planner.PipelineRel; @@ -27,6 +29,7 @@ import io.kubernetes.client.openapi.models.V1OwnerReference; import com.linkedin.hoptimator.Deployer; +import com.linkedin.hoptimator.PendingDelete; import com.linkedin.hoptimator.Trigger; import com.linkedin.hoptimator.UserJob; import com.linkedin.hoptimator.Validated; @@ -47,6 +50,7 @@ import com.linkedin.hoptimator.jdbc.HoptimatorConnection; import com.linkedin.hoptimator.jdbc.HoptimatorDriver; import com.linkedin.hoptimator.jdbc.HoptimatorDdlUtils; +import com.linkedin.hoptimator.jdbc.ValidationService; /** @@ -67,6 +71,7 @@ public class LogicalTableDeployer implements Deployer, Validated { private final Properties tierProps; private final K8sContext context; private final K8sApi databasesApi; + private final K8sApi logicalTableApi; private final List tierDeployers = new ArrayList<>(); private final List pipelineDeployers = new ArrayList<>(); @@ -83,16 +88,20 @@ public class LogicalTableDeployer implements Deployer, Validated { private Map cachedTierSources; LogicalTableDeployer(Source source, Properties tierProps, K8sContext context) { - this(source, tierProps, context, new K8sApi<>(context, K8sApiEndpoints.DATABASES)); + this(source, tierProps, context, + new K8sApi<>(context, K8sApiEndpoints.DATABASES), + new K8sApi<>(context, K8sApiEndpoints.LOGICAL_TABLES)); } - /** Package-private constructor for testing — accepts an injectable database API. */ + /** Package-private constructor for testing — accepts injectable K8s APIs. */ LogicalTableDeployer(Source source, Properties tierProps, K8sContext context, - K8sApi databasesApi) { + K8sApi databasesApi, + K8sApi logicalTableApi) { this.source = source; this.tierProps = tierProps; this.context = context; this.databasesApi = databasesApi; + this.logicalTableApi = logicalTableApi; } /** @@ -113,7 +122,7 @@ K8sLogicalTableDeployer createLogicalTableDeployer( *

Called by {@link com.linkedin.hoptimator.jdbc.ValidationService} before deployment. */ @Override - public void validate(Validator.Issues issues) { + public void validate(Validator.Issues issues, Connection connection) { try { // Pre-register the row type in tier schemas so deployers (e.g. VeniceDeployer) can // call HoptimatorDriver.rowType() during their own validate() calls. @@ -124,7 +133,7 @@ public void validate(Validator.Issues issues) { Collection deployers = DeploymentService.deployers(tierSource, context.connection()); for (Deployer deployer : deployers) { if (deployer instanceof Validated) { - ((Validated) deployer).validate(issues); + ((Validated) deployer).validate(issues, connection); } } } @@ -206,15 +215,76 @@ private void deployAll(boolean update) throws SQLException { } } + /** + * Looks up the existing LogicalTable CRD's UID, used as the {@code selfOwnerUid} on + * {@link PendingDelete} so the pre-delete dep check excludes pipelines owned by this CRD + * (the implicit inter-tier pipelines are cascade-deleted with it). Returns {@code null} if + * the CRD doesn't exist (pre-create or already-deleted state). + */ + private String existingLogicalTableUid() throws SQLException { + if (logicalTableApi == null) { + return null; + } + String crdName = K8sUtils.canonicalizeName(source.path()); + V1alpha1LogicalTable existing = logicalTableApi.getIfExists(context.namespace(), crdName); + if (existing == null || existing.getMetadata() == null) { + return null; + } + return existing.getMetadata().getUid(); + } + + /** + * Deletes a logical table. + * + *

A logical DROP is structurally equivalent to running DROP TABLE on each tier plus + * deleting the LogicalTable CRD. We mirror that shape exactly: each tier goes through the + * same {@code validateOrThrow → DeploymentService.delete} pipeline a standalone DROP would. + * The {@link PendingDelete}'s {@code selfOwnerUid} is the LogicalTable CRD's UID so that the + * implicit inter-tier pipelines (owned by the CRD, cascade-deleted with it) are excluded + * from the dependent set — only external pipelines block. + * + *

    + *
  1. Per-tier dep check via the validator framework. Any active external pipeline blocks. + *
  2. Delete the {@code LogicalTable} CRD. K8s owner-ref cascade removes its implicit + * inter-tier Pipelines and their Flink/YAML children. Must succeed. + *
  3. Per-tier physical cleanup (Kafka topic, Venice store, ...). Best effort — a + * stranded tier resource is recoverable; aborting mid-DROP isn't. + *
  4. Per-tier schema cleanup (deregister the {@code TemporaryTable} in tier schemas). + *
+ */ @Override public void delete() throws SQLException { - // TODO: Implement safe logical table deletion. - // Deletion is blocked until we can verify no active pipelines depend on this table. - // See: LogicalTableDeployer.delete() - throw new SQLFeatureNotSupportedException( - "Logical table deletion is not yet supported. " - + "Cannot safely delete physical tier resources without verifying no active pipelines " - + "depend on this table."); + Map tierSources = buildTierSources(); + HoptimatorConnection conn = context.connection(); + String selfUid = existingLogicalTableUid(); + + // 1. Per-tier pre-flight dep check. + for (Source tierSource : tierSources.values()) { + ValidationService.validateOrThrow(new PendingDelete<>(tierSource, selfUid), conn); + } + + // 2. Delete the LogicalTable CRD (cascades owned pipelines/triggers). + createLogicalTableDeployer( + K8sUtils.canonicalizeName(source.path()), source.database(), buildTierMap()).delete(); + + // 3. Per-tier physical cleanup. Best-effort: only deregister a tier's schema entry when its + // physical delete succeeded; failed tiers keep their entries so the user can retry. + for (Source tierSource : tierSources.values()) { + boolean tierSucceeded = true; + for (Deployer deployer : DeploymentService.deployers(tierSource, conn)) { + try { + deployer.delete(); + } catch (Exception e) { + tierSucceeded = false; + log.warn("Tier cleanup failed for {} (continuing): {}", + tierSource.pathString(), e.getMessage(), e); + } + } + if (tierSucceeded) { + HoptimatorDdlUtils.removeTableFromSchema(conn, + tierSource.catalog(), tierSource.schema(), tierSource.table()); + } + } } @Override @@ -445,7 +515,8 @@ void deployPipelineBundle(String fromTier, String toTier, Map hoptimatorDriverMock; + @Mock + MockedStatic validationServiceMock; + @Mock K8sLogicalTableDeployer mockCrdDeployer; @@ -108,9 +119,11 @@ private static Properties twoTierProps(String nearlineDb, String offlineDb) { private static K8sContext mockContext() { K8sContext ctx = mock(K8sContext.class); - when(ctx.namespace()).thenReturn("default"); - when(ctx.withOwner(any())).thenReturn(ctx); - when(ctx.withLabel(anyString(), anyString())).thenReturn(ctx); + // Each stub is used by some tests but not all — mark lenient individually so the class + // doesn't need @MockitoSettings(strictness = Strictness.LENIENT). + lenient().when(ctx.namespace()).thenReturn("default"); + lenient().when(ctx.withOwner(any())).thenReturn(ctx); + lenient().when(ctx.withLabel(anyString(), anyString())).thenReturn(ctx); return ctx; } @@ -126,7 +139,9 @@ private static Source testSource() { private LogicalTableDeployer deployerWithMockCrd( Source src, Properties props, K8sContext ctx, FakeK8sApi dbApi) { - return new LogicalTableDeployer(src, props, ctx, dbApi) { + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); + return new LogicalTableDeployer(src, props, ctx, dbApi, ltApi) { @Override K8sLogicalTableDeployer createLogicalTableDeployer( String crdName, String databaseLabel, Map tierMap) { @@ -228,17 +243,156 @@ void pipelineNameNearlineToOnline() { LogicalTableDeployer.pipelineName("events", "nearline", "online")); } - // delete() / specify() tests + // delete() / DependencyGuarded / specify() tests + + /** Builds a 2-tier deployer with mocked CRD deployer and pre-populated fake K8s APIs. */ + private LogicalTableDeployer deployerWithApis(Properties props, + List dbs, List logicalTables) { + FakeK8sApi dbApi = new FakeK8sApi<>(new ArrayList<>(dbs)); + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>(logicalTables)); + return new LogicalTableDeployer(testSource(), props, mockContext(), dbApi, ltApi) { + @Override + K8sLogicalTableDeployer createLogicalTableDeployer( + String crdName, String databaseLabel, Map tierMap) { + return mockCrdDeployer; + } + }; + } @Test - void deleteThrowsSQLFeatureNotSupportedException() { - Properties props = new Properties(); - props.setProperty("nearline", "kafka-db"); - props.setProperty("online", "venice-db"); + void deleteThrowsWhenCrdDeleteFails() throws SQLException { + LogicalTableDeployer deployer = deployerWithApis( + twoTierProps("kafka-db", "venice-db"), + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), + Collections.emptyList()); + + doThrow(new SQLException("crd gone")).when(mockCrdDeployer).delete(); + + SQLException ex = assertThrows(SQLException.class, deployer::delete); + assertTrue(ex.getMessage().contains("crd gone")); + verify(mockCrdDeployer).delete(); + // Tier deployers must not run when the CRD delete itself fails. + deploymentServiceMock.verify( + () -> DeploymentService.deployers(any(), any()), never()); + } + + @Test + void deleteSwallowsTierDeleteFailuresAndContinues() throws SQLException { + LogicalTableDeployer deployer = deployerWithApis( + twoTierProps("kafka-db", "venice-db"), + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), + Collections.emptyList()); + + // Two tier deployers: first throws, second succeeds. The overall delete must return cleanly. + Deployer failing = mock(Deployer.class); + Deployer succeeding = mock(Deployer.class); + doThrow(new SQLException("kafka delete failed")).when(failing).delete(); + + deploymentServiceMock.when(() -> DeploymentService.deployers(any(), any())) + .thenReturn(Collections.singletonList(failing), Collections.singletonList(succeeding)); + + // Must NOT throw despite the failing tier. + deployer.delete(); + + verify(mockCrdDeployer).delete(); + verify(failing).delete(); + verify(succeeding).delete(); + } + + /** + * Executes {@code body} with {@link HoptimatorDdlUtils} statics stubbed, then gives the + * supplied verifier a handle on the mock to assert side effects. + * + *

Uses try-with-resources rather than the project-standard {@code @Mock MockedStatic} + * field because sibling tests in this class rely on the real static methods of + * {@code HoptimatorDdlUtils} (via {@code ensureTierRowTypesRegistered} and friends); a + * class-level mock would intercept them and break unrelated tests. + */ + private void withMockedDdlUtils(Runnable body, Consumer> verifier) { + try (MockedStatic utilsMock = mockStatic(HoptimatorDdlUtils.class)) { + body.run(); + verifier.accept(utilsMock); + } + } + + @Test + void deleteRemovesTierEntriesFromConnectionSchema() throws SQLException { + LogicalTableDeployer deployer = deployerWithApis( + twoTierProps("kafka-db", "venice-db"), + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), + Collections.emptyList()); + + Deployer tierDeployer = mock(Deployer.class); + deploymentServiceMock.when(() -> DeploymentService.deployers(any(), any())) + .thenReturn(Collections.singletonList(tierDeployer)); + + withMockedDdlUtils(() -> { + try { + deployer.delete(); + } catch (SQLException e) { + throw new RuntimeException(e); + } + }, utilsMock -> { + // Inverse of the registerTemporaryTableInSchema calls that ran at create time — one + // call per tier source (null catalog, non-null schema = tier "KAFKA" / "VENICE"). + utilsMock.verify(() -> HoptimatorDdlUtils.removeTableFromSchema( + any(), any(), eq("KAFKA"), any())); + utilsMock.verify(() -> HoptimatorDdlUtils.removeTableFromSchema( + any(), any(), eq("VENICE"), any())); + }); + } + + @Test + void deleteKeepsSchemaEntryForTierWhoseDeleteFailed() throws SQLException { + // Two tiers: the first (kafka-db → KAFKA) fails to delete; the second succeeds. + // The failed tier's schema entry must NOT be removed; the succeeded tier's must be. + LogicalTableDeployer deployer = deployerWithApis( + twoTierProps("kafka-db", "venice-db"), + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), + Collections.emptyList()); + + Deployer failingTier = mock(Deployer.class); + Deployer succeedingTier = mock(Deployer.class); + doThrow(new SQLException("kafka delete failed")).when(failingTier).delete(); + + // DeploymentService.deployers is invoked once per tier — order follows tierSources. + deploymentServiceMock.when(() -> DeploymentService.deployers(any(), any())) + .thenReturn(Collections.singletonList(failingTier), + Collections.singletonList(succeedingTier)); + + withMockedDdlUtils(() -> { + try { + deployer.delete(); + } catch (SQLException e) { + throw new RuntimeException(e); + } + }, utilsMock -> { + // KAFKA failed → its schema entry must NOT be removed. + utilsMock.verify(() -> HoptimatorDdlUtils.removeTableFromSchema( + any(), any(), eq("KAFKA"), any()), never()); + // VENICE succeeded → its schema entry is removed. + utilsMock.verify(() -> HoptimatorDdlUtils.removeTableFromSchema( + any(), any(), eq("VENICE"), any())); + }); + } + + @Test + void deleteRunsCrdDeleteBeforeTierDeletes() throws SQLException { + LogicalTableDeployer deployer = deployerWithApis( + twoTierProps("kafka-db", "venice-db"), + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), + Collections.emptyList()); + + Deployer tierDeployer = mock(Deployer.class); + deploymentServiceMock.when(() -> DeploymentService.deployers(any(), any())) + .thenReturn(Collections.singletonList(tierDeployer)); + + deployer.delete(); - LogicalTableDeployer deployer = new LogicalTableDeployer(makeSource("mydb", "myTable"), props, null); - SQLFeatureNotSupportedException e = assertThrows(SQLFeatureNotSupportedException.class, deployer::delete); - assertTrue(e.getMessage().contains("Logical table deletion is not yet supported")); + InOrder inOrder = inOrder(mockCrdDeployer, tierDeployer); + inOrder.verify(mockCrdDeployer).delete(); + inOrder.verify(tierDeployer, atLeastOnce()).delete(); } // CRD model construction tests @@ -490,7 +644,9 @@ void createWithNearlineAndOnlineTiersAttemptsPipelineDeployment() throws Excepti // Use a subclass that mocks the CRD deployer but does NOT suppress deployPipelineBundle, // so the pipeline path is exercised and fails due to the null connection in mockContext(). - LogicalTableDeployer deployer = new LogicalTableDeployer(testSource(), props, mockContext(), dbApi) { + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); + LogicalTableDeployer deployer = new LogicalTableDeployer(testSource(), props, mockContext(), dbApi, ltApi) { @Override K8sLogicalTableDeployer createLogicalTableDeployer( String crdName, String databaseLabel, Map tierMap) { @@ -513,21 +669,25 @@ void validateSucceedsWithValidTierConfiguration() throws Exception { FakeK8sApi dbApi = new FakeK8sApi<>(Arrays.asList(makeDb("nearline-db", "NEARLINE"), makeDb("offline-db", "OFFLINE"))); + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); Validator.Issues issues = new Validator.Issues("test"); new LogicalTableDeployer( - testSource(), twoTierProps("nearline-db", "offline-db"), mockContext(), dbApi) - .validate(issues); + testSource(), twoTierProps("nearline-db", "offline-db"), mockContext(), dbApi, ltApi) + .validate(issues, null); assertTrue(issues.valid()); } @Test void validateReportsIssueWhenDatabaseNotFound() throws Exception { + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); Validator.Issues issues = new Validator.Issues("test"); new LogicalTableDeployer( testSource(), twoTierProps("missing-db", "also-missing"), - mockContext(), new FakeK8sApi<>(new ArrayList<>())) - .validate(issues); + mockContext(), new FakeK8sApi<>(new ArrayList<>()), ltApi) + .validate(issues, null); assertFalse(issues.valid()); } @@ -544,13 +704,15 @@ void validateCallsValidatedDeployersWhenTiersExist() throws Exception { new FakeK8sApi<>(Arrays.asList(makeDb("nearline-db", "NEARLINE"))); K8sContext ctx = mock(K8sContext.class); + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); Validator.Issues issues = new Validator.Issues("test"); new LogicalTableDeployer( - makeSource("logical", "testevent"), oneTierProps, ctx, dbApi) - .validate(issues); + makeSource("logical", "testevent"), oneTierProps, ctx, dbApi, ltApi) + .validate(issues, null); - verify(mockValidatedDeployer).validate(issues); + verify(mockValidatedDeployer).validate(issues, null); assertTrue(issues.valid()); } @@ -591,10 +753,12 @@ void ensureTierRowTypesRegisteredWithConnectionRecordsRowTypeError() throws Exce FakeK8sApi dbApi = new FakeK8sApi<>(Arrays.asList(makeDb("nearline-db", "NEARLINE"))); + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); Validator.Issues issues = new Validator.Issues("test"); new LogicalTableDeployer( - makeSource("logical", "testevent"), oneTierProps, ctx, dbApi) - .validate(issues); + makeSource("logical", "testevent"), oneTierProps, ctx, dbApi, ltApi) + .validate(issues, null); assertFalse(issues.valid()); } @@ -609,7 +773,9 @@ void specifyWithNearlineAndOnlineThrowsException() { Properties props = twoTierProps("nearline-db", "online-db"); props.setProperty(LogicalTier.ONLINE.tierName(), "online-db"); - assertThrows(Exception.class, () -> new LogicalTableDeployer(testSource(), props, mockContext(), dbApi).specify()); + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); + assertThrows(Exception.class, () -> new LogicalTableDeployer(testSource(), props, mockContext(), dbApi, ltApi).specify()); } @Test @@ -620,8 +786,10 @@ void specifyWithOfflineTierOnlyDoesNotAttemptPipeline() throws Exception { Properties props = new Properties(); props.setProperty(LogicalTier.OFFLINE.tierName(), "offline-db"); + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); List specs = new LogicalTableDeployer( - testSource(), props, mockContext(), dbApi).specify(); + testSource(), props, mockContext(), dbApi, ltApi).specify(); assertNotNull(specs); assertTrue(specs.isEmpty(), "offline-only — no pipeline spec should be attempted"); @@ -642,9 +810,11 @@ void specifyIncludesTierResourceSpecsFromDeploymentService() throws Exception { // specify() calls DeploymentService.specify() per tier before the pipeline path, // which fails (null connection) — so we only see tier specs, not job specs. + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); List specs; try { - specs = new LogicalTableDeployer(testSource(), props, mockContext(), dbApi).specify(); + specs = new LogicalTableDeployer(testSource(), props, mockContext(), dbApi, ltApi).specify(); } catch (SQLException ignored) { // Pipeline planning may throw due to null connection; tier specs are added first. return; @@ -659,9 +829,11 @@ void specifyWithNearlineAndOfflineThrowsExceptionOnPipelinePlanning() { FakeK8sApi dbApi = new FakeK8sApi<>(Arrays.asList(makeDb("nearline-db", "NEARLINE"), makeDb("offline-db", "OFFLINE"))); + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); assertThrows(Exception.class, () -> new LogicalTableDeployer(testSource(), twoTierProps("nearline-db", "offline-db"), - mockContext(), dbApi).specify()); + mockContext(), dbApi, ltApi).specify()); } @Test @@ -705,7 +877,9 @@ private LogicalTableDeployer deployerWithJobTemplates( new FakeK8sApi<>(jobTemplates); FakeK8sApi triggerApi = new FakeK8sApi<>(preExistingTriggers); - return new LogicalTableDeployer(src, props, ctx, dbApi) { + FakeK8sApi ltApi = + new FakeK8sApi<>(new ArrayList<>()); + return new LogicalTableDeployer(src, props, ctx, dbApi, ltApi) { @Override K8sLogicalTableDeployer createLogicalTableDeployer( String crdName, String databaseLabel, Map tierMap) { diff --git a/hoptimator-logical/src/test/resources/logical-ddl.id b/hoptimator-logical/src/test/resources/logical-ddl.id index 7284ae72..b58d54e1 100644 --- a/hoptimator-logical/src/test/resources/logical-ddl.id +++ b/hoptimator-logical/src/test/resources/logical-ddl.id @@ -168,14 +168,7 @@ Failed to generate key schema for Venice store testevent !describe "VENICE"."testevent" # ───────────────────────────────────────────────────────────────────────────── -# Test 7: DROP TABLE — disabled; should fail with helpful error message -# ───────────────────────────────────────────────────────────────────────────── -drop table "LOGICAL"."testevent"; -Logical table deletion is not yet supported -!error - -# ───────────────────────────────────────────────────────────────────────────── -# Test 8: CREATE TABLE against non-existent schema is rejected +# Test 7: CREATE TABLE against non-existent schema is rejected # ───────────────────────────────────────────────────────────────────────────── create table "LOGICAL-NONEXISTENT"."testevent" ("KEY" varchar, "id" bigint); Schema for LOGICAL-NONEXISTENT.testevent not found. @@ -220,8 +213,58 @@ spec: !specify create-table-test # ───────────────────────────────────────────────────────────────────────────── -# Clean up (deletion not yet supported — verified in Test 6 above) +# Test 9: Dependency guard — an external MV consuming some logical table tier +# must block DROP TABLE on the logical table. The same depends-on labels that +# protect plain tier drops also protect the composite logical-table drop, since +# LogicalTableDeployer's guardedResources() exposes every tier source to the framework. # ───────────────────────────────────────────────────────────────────────────── -drop table "LOGICAL"."pageview"; -Logical table deletion is not yet supported +create or replace materialized view VENICE."testevent$guard" as select "KEY" as "KEY", "VALUE" as "memberId" from KAFKA."existing-topic-1"; +(0 rows modified) + +!update + +# Drop is blocked — testevent$guard's pipeline depends on VENICE/testevent, +# which is LOGICAL.testevent's online tier. +drop table "LOGICAL"."testevent"; +active pipeline(s) depend on it !error + +# Drop the dependent MV first to release the label. +drop materialized view VENICE."testevent$guard"; +(0 rows modified) + +!update + +# ───────────────────────────────────────────────────────────────────────────── +# Test 10: DROP TABLE — cascades to the LogicalTable CRD and its implicit +# inter-tier pipeline; tier resources are best-effort cleaned up afterward. +# ───────────────────────────────────────────────────────────────────────────── +drop table "LOGICAL"."testevent"; +(0 rows modified) + +!update + +# The owner-ref cascade removes the implicit nearline→online pipeline. +select name from "k8s".pipelines where name = 'logical-testevent-nearline-to-online'; ++------+ +| NAME | ++------+ ++------+ +(0 rows) + +!ok + +drop table "LOGICAL"."pageview"; +(0 rows modified) + +!update + +# Verify the pageview pipeline is gone too. +select name from "k8s".pipelines where name = 'logical-pageview-nearline-to-online'; ++------+ +| NAME | ++------+ ++------+ +(0 rows) + +!ok diff --git a/hoptimator-logical/src/test/resources/logical-offline-ddl.id b/hoptimator-logical/src/test/resources/logical-offline-ddl.id index 82fcd538..9e154ba2 100644 --- a/hoptimator-logical/src/test/resources/logical-offline-ddl.id +++ b/hoptimator-logical/src/test/resources/logical-offline-ddl.id @@ -77,3 +77,8 @@ select paused from "k8s".table_triggers where name = 'logical-members-offline-tr (1 row) !ok + +drop table "LOGICAL_OFFLINE"."MEMBERS"; +(0 rows modified) + +!update \ No newline at end of file From 856d5e448bfe2a53243dddd3d834c48a3faf126a Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Thu, 30 Apr 2026 21:37:50 -0400 Subject: [PATCH 03/15] test: integration scenarios + cleanup test warnings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit kafka-ddl-create-table.id: cross-driver dependency-guard scenarios exercising the new pre-delete check end-to-end through the kafka driver — drop-table-while-pipeline-depends-on-it (source side and partial-view sink side). The bulk of the file count is mechanical noise reduction across existing test files: dropped unused imports, tightened generics on @SuppressWarnings, etc. — fallout from the warning_cleanup pass. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../k8s/K8sApiErrorResponseTest.java | 1 + .../hoptimator/k8s/K8sSourceDeployerTest.java | 25 +++++++------ .../test/resources/kafka-ddl-create-table.id | 37 ++++++++++++++++++- .../hoptimator/mysql/MySqlDriverTest.java | 3 -- .../hoptimator/venice/ClusterSchemaTest.java | 3 -- 5 files changed, 50 insertions(+), 19 deletions(-) diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sApiErrorResponseTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sApiErrorResponseTest.java index 1d753c42..ff9cd7f9 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sApiErrorResponseTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sApiErrorResponseTest.java @@ -78,6 +78,7 @@ void deleteThrowsWhenResponseIsErrorStatus() throws ApiException { } @Test + @SuppressWarnings("unchecked") void updateThrowsWhenResponseIsErrorStatusOnFinalUpdate() throws ApiException { V1alpha1Pipeline pipeline = makePipeline("bad-pipeline", "test-ns"); V1alpha1Pipeline existing = makePipeline("bad-pipeline", "test-ns"); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sSourceDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sSourceDeployerTest.java index 4535a359..9bc94560 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sSourceDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sSourceDeployerTest.java @@ -5,7 +5,6 @@ import com.linkedin.hoptimator.k8s.models.V1alpha1TableTemplate; import com.linkedin.hoptimator.k8s.models.V1alpha1TableTemplateList; import com.linkedin.hoptimator.k8s.models.V1alpha1TableTemplateSpec; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.kubernetes.client.openapi.models.V1ObjectMeta; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -26,13 +25,10 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -@SuppressFBWarnings(value = {"RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"}, - justification = "Mockito doReturn().when() stubs — framework captures the return value") class K8sSourceDeployerTest { @Mock @@ -58,7 +54,12 @@ K8sYamlApi createYamlApi(K8sContext context) { return fakeYamlApi; } }; + } + + /** Wires up {@code mockContext.connection()} for tests that exercise specify()'s template path. */ + private void stubConnection() { when(mockContext.connection()).thenReturn(connection); + when(connection.connectionProperties()).thenReturn(new Properties()); } private K8sSourceDeployer makeDeployer(Source source) { @@ -85,7 +86,7 @@ K8sSnapshot createSnapshot(K8sContext context) { @Test void specifyWithNoTemplatesReturnsEmpty() throws SQLException { - doReturn(new Properties()).when(connection).connectionProperties(); + stubConnection(); Source source = new Source("testdb", Arrays.asList("schema", "table"), Collections.emptyMap()); @@ -100,7 +101,7 @@ void specifyWithNoTemplatesReturnsEmpty() throws SQLException { @Test void specifyRendersMatchingTemplate() throws SQLException { - doReturn(new Properties()).when(connection).connectionProperties(); + stubConnection(); templates.add(new V1alpha1TableTemplate() .metadata(new V1ObjectMeta().name("template1")) @@ -121,7 +122,7 @@ void specifyRendersMatchingTemplate() throws SQLException { @Test void specifyFiltersOutNonMatchingDatabases() throws SQLException { - doReturn(new Properties()).when(connection).connectionProperties(); + stubConnection(); templates.add(new V1alpha1TableTemplate() .metadata(new V1ObjectMeta().name("template1")) @@ -141,7 +142,7 @@ void specifyFiltersOutNonMatchingDatabases() throws SQLException { @Test void specifyWithJobPropertiesInOptions() throws SQLException { - doReturn(new Properties()).when(connection).connectionProperties(); + stubConnection(); templates.add(new V1alpha1TableTemplate() .metadata(new V1ObjectMeta().name("template1")) @@ -161,7 +162,7 @@ void specifyWithJobPropertiesInOptions() throws SQLException { @Test void specifyWithNullDatabasesMatchesAll() throws SQLException { - doReturn(new Properties()).when(connection).connectionProperties(); + stubConnection(); templates.add(new V1alpha1TableTemplate() .metadata(new V1ObjectMeta().name("template1")) @@ -182,7 +183,7 @@ void specifyWithNullDatabasesMatchesAll() throws SQLException { @Test void specifyRendersNonEmptyYamlWithSourceContent() throws SQLException { // Verify fields are non-empty. - doReturn(new Properties()).when(connection).connectionProperties(); + stubConnection(); templates.add(new V1alpha1TableTemplate() .metadata(new V1ObjectMeta().name("template1")) @@ -206,7 +207,7 @@ void specifyRendersNonEmptyYamlWithSourceContent() throws SQLException { @Test void getJobPropertiesFromOptionsMapsCorrectKeys() throws SQLException { - doReturn(new Properties()).when(connection).connectionProperties(); + stubConnection(); // Template uses {{job.properties}} prefix variable to expose job properties templates.add(new V1alpha1TableTemplate() @@ -233,7 +234,7 @@ void getJobPropertiesFromOptionsMapsCorrectKeys() throws SQLException { @Test void getJobPropertiesFromOptionsFiltersNonMatchingKeys() throws SQLException { // Ensures the filter actually filters — only job.properties.* keys should be mapped - doReturn(new Properties()).when(connection).connectionProperties(); + stubConnection(); templates.add(new V1alpha1TableTemplate() .metadata(new V1ObjectMeta().name("template1")) diff --git a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id index 32efc51a..b3ba2c09 100644 --- a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id +++ b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id @@ -84,7 +84,42 @@ spec: segment.bytes: 1073741824 !specify create-table-test -# Clean up - drop table +# ───────────────────────────────────────────────────────────────────────────── +# Dependency guard: a Kafka topic referenced by an MV's pipeline — as either a +# source or a sink — cannot be dropped until the dependent MV is removed. +# Exercises the DependencyGuarded SPI across deployer types: the label-based +# check protects Kafka topic drops that would orphan an active downstream +# pipeline. +# +# A single MV exercises both edges. The MV is a partial view +# (KAFKA."create-table-test$guard") whose implicit sink falls back to +# create-table-test, and reads from existing-topic-2. +# ───────────────────────────────────────────────────────────────────────────── + +# Pipeline reads from existing-topic-2 (source) and writes to create-table-test +# (partial-view sink). Its Pipeline CRD gets a depends-on-* label per edge. +create or replace materialized view KAFKA."create-table-test$guard" as select * from KAFKA."existing-topic-2"; +(0 rows modified) + +!update + +# Source-side guard: existing-topic-2 is a source of the MV's pipeline. +drop table KAFKA."existing-topic-2"; +active pipeline(s) depend on it +!error + +# Sink-side guard: create-table-test is the partial-view sink of the MV's pipeline. +drop table KAFKA."create-table-test"; +active pipeline(s) depend on it +!error + +# Drop the dependent MV first; its pipeline (and both labels) go away. +drop materialized view KAFKA."create-table-test$guard"; +(0 rows modified) + +!update + +# Now sink drop should succeed drop table KAFKA."create-table-test"; (0 rows modified) diff --git a/hoptimator-mysql/src/test/java/com/linkedin/hoptimator/mysql/MySqlDriverTest.java b/hoptimator-mysql/src/test/java/com/linkedin/hoptimator/mysql/MySqlDriverTest.java index 8f434bf4..ad9534c1 100644 --- a/hoptimator-mysql/src/test/java/com/linkedin/hoptimator/mysql/MySqlDriverTest.java +++ b/hoptimator-mysql/src/test/java/com/linkedin/hoptimator/mysql/MySqlDriverTest.java @@ -1,6 +1,5 @@ package com.linkedin.hoptimator.mysql; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.calcite.jdbc.CalciteConnection; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.impl.AbstractSchema; @@ -25,8 +24,6 @@ @ExtendWith(MockitoExtension.class) -@SuppressFBWarnings(value = {"OBL_UNSATISFIED_OBLIGATION", "ODR_OPEN_DATABASE_RESOURCE", "DMI_EMPTY_DB_PASSWORD"}, - justification = "Mock objects do not hold real resources") class MySqlDriverTest { /** Returns a driver whose {@code createMySqlRootSchema()} yields a no-op schema. */ diff --git a/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/ClusterSchemaTest.java b/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/ClusterSchemaTest.java index f033b5d1..c3db472d 100644 --- a/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/ClusterSchemaTest.java +++ b/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/ClusterSchemaTest.java @@ -6,7 +6,6 @@ import com.linkedin.venice.controllerapi.MultiStoreResponse; import com.linkedin.venice.exceptions.ErrorType; import com.linkedin.venice.security.SSLFactory; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.calcite.schema.Table; import org.apache.calcite.schema.lookup.LikePattern; import org.apache.calcite.schema.lookup.Lookup; @@ -31,8 +30,6 @@ @ExtendWith(MockitoExtension.class) -@SuppressFBWarnings(value = {"OBL_UNSATISFIED_OBLIGATION", "ODR_OPEN_DATABASE_RESOURCE"}, - justification = "Mock objects created in stubbing setup don't need resource management") class ClusterSchemaTest { @Mock From b345f1c66d5bef0f558e3835cba0369f9f57169d Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Fri, 1 May 2026 13:14:01 -0400 Subject: [PATCH 04/15] update integration test --- hoptimator-logical/src/test/resources/logical-ddl.id | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/hoptimator-logical/src/test/resources/logical-ddl.id b/hoptimator-logical/src/test/resources/logical-ddl.id index b58d54e1..afbe1176 100644 --- a/hoptimator-logical/src/test/resources/logical-ddl.id +++ b/hoptimator-logical/src/test/resources/logical-ddl.id @@ -218,6 +218,12 @@ spec: # protect plain tier drops also protect the composite logical-table drop, since # LogicalTableDeployer's guardedResources() exposes every tier source to the framework. # ───────────────────────────────────────────────────────────────────────────── + +# Drop is blocked — logical table inner pipeline depends on KAFKA/testevent +drop table "KAFKA"."testevent"; +active pipeline(s) depend on it +!error + create or replace materialized view VENICE."testevent$guard" as select "KEY" as "KEY", "VALUE" as "memberId" from KAFKA."existing-topic-1"; (0 rows modified) From 4be1a8b4abbb180f1ab4c7358d37da747662270d Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Fri, 1 May 2026 13:28:50 -0400 Subject: [PATCH 05/15] comment cleanups and add @Nullable annotation --- hoptimator-api/build.gradle | 3 ++- .../linkedin/hoptimator/PendingDelete.java | 7 +++--- .../com/linkedin/hoptimator/Validated.java | 3 +-- .../hoptimator/ValidatorProvider.java | 5 +---- .../jdbc/HoptimatorDdlExecutor.java | 3 --- .../k8s/K8sMaterializedViewDeployer.java | 8 +------ .../k8s/K8sPipelineDependencyValidator.java | 10 +++------ .../hoptimator/k8s/K8sPipelineDeployer.java | 6 +---- .../hoptimator/k8s/K8sValidatorProvider.java | 2 -- .../k8s/PipelineDependencyChecker.java | 8 ++++--- .../k8s/K8sPipelineDeployerTest.java | 22 ++++--------------- .../test/resources/kafka-ddl-create-table.id | 5 ++--- .../src/test/resources/logical-ddl.id | 7 +++--- 13 files changed, 27 insertions(+), 62 deletions(-) diff --git a/hoptimator-api/build.gradle b/hoptimator-api/build.gradle index 571a1964..d73eca13 100644 --- a/hoptimator-api/build.gradle +++ b/hoptimator-api/build.gradle @@ -4,7 +4,8 @@ plugins { } dependencies { - // plz keep it this way + // This package should have minimal dependencies + compileOnly 'com.google.code.findbugs:jsr305:3.0.2' } publishing { diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java index d4d77201..3c93f993 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java @@ -1,10 +1,11 @@ package com.linkedin.hoptimator; import java.util.Objects; +import javax.annotation.Nullable; /** - * A type-tagged wrapper signalling that {@code target} is about to be deleted. Pre-delete + * A type-tagged wrapper signaling that {@code target} is about to be deleted. Pre-delete * validators (e.g. dependency guards that block DROP TABLE when a pipeline still references * the resource) key off this wrapper rather than the raw target type — so an unrelated future * caller of {@code ValidationService.validateOrThrow(source, connection)} doesn't accidentally @@ -23,7 +24,7 @@ public PendingDelete(T target) { this(target, null); } - public PendingDelete(T target, String selfOwnerUid) { + public PendingDelete(T target, @Nullable String selfOwnerUid) { this.target = Objects.requireNonNull(target, "target"); this.selfOwnerUid = selfOwnerUid; } @@ -33,7 +34,7 @@ public T target() { } /** UID of the K8s resource whose owned objects should be excluded from the dependent set. */ - public String selfOwnerUid() { + public @Nullable String selfOwnerUid() { return selfOwnerUid; } diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validated.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validated.java index 12d21537..7c32c148 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validated.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Validated.java @@ -8,8 +8,7 @@ public interface Validated { /** * Validates {@code this}, recording any problems in {@code issues}. The connection is always * supplied so validators can run lookups against external systems (e.g. pre-delete dependency - * checks). Pass {@code null} only when the caller genuinely has no connection — most validators - * ignore it, but some require it. + * checks). */ void validate(Validator.Issues issues, Connection connection); } diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java index 5a4b4d7f..d4729e4a 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/ValidatorProvider.java @@ -7,10 +7,7 @@ public interface ValidatorProvider { /** - * Returns validators that should be applied to {@code obj}. The connection is always supplied - * — providers that don't need it can ignore it (matches the {@code DeployerProvider} pattern). - * Implementations capturing the connection in returned validators must accept that the - * connection may be {@code null} when the caller has none. + * Returns validators that should be applied to {@code obj}. */ Collection validators(T obj, Connection connection); } diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index d1574563..18e229e7 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -19,7 +19,6 @@ */ package com.linkedin.hoptimator.jdbc; -import com.linkedin.hoptimator.Database; import com.linkedin.hoptimator.Deployer; import com.linkedin.hoptimator.PendingDelete; import com.linkedin.hoptimator.Source; @@ -396,8 +395,6 @@ public void execute(SqlDropObject drop, CalcitePrepare.Context context) { final List schemaPath = pair.left.path(null); List tablePath = new ArrayList<>(schemaPath); tablePath.add(tableName); - String database = pair.left.schema instanceof Database - ? ((Database) pair.left.schema).databaseName() : null; Collection deployers = null; try { diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployer.java index 7cd3ebff..76967411 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sMaterializedViewDeployer.java @@ -14,13 +14,7 @@ import java.util.List; -/** - * Deploys View and Pipeline objects, along with all the pipeline elements. - * - *

The dependency guard is implemented on {@link K8sViewDeployer} (which is what gets - * routed to during DROP MV), so this class doesn't need to implement - * {@link com.linkedin.hoptimator.DependencyGuarded} itself. - */ +/** Deploys View and Pipeline objects, along with all the pipeline elements. */ class K8sMaterializedViewDeployer implements Deployer { private final MaterializedView view; diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java index b27e490c..5c17e2ee 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java @@ -2,6 +2,7 @@ import java.sql.Connection; import java.sql.SQLException; +import javax.annotation.Nullable; import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.Validator; @@ -11,25 +12,20 @@ * Pre-delete dependency check, run by the validator framework when a {@link Source} is wrapped * in {@link com.linkedin.hoptimator.PendingDelete}. Delegates to the existing * {@link PipelineDependencyChecker} (label-selector + annotation collision-guard + self-uid - * exclusion) and surfaces any blocking pipeline as a validation error rather than throwing - * directly — that's what the validator contract calls for. + * exclusion) and surfaces any blocking pipeline as a validation error */ final class K8sPipelineDependencyValidator implements Validator { private final Source source; private final String selfOwnerUid; - K8sPipelineDependencyValidator(Source source, String selfOwnerUid) { + K8sPipelineDependencyValidator(Source source, @Nullable String selfOwnerUid) { this.source = source; this.selfOwnerUid = selfOwnerUid; } @Override public void validate(Issues issues, Connection connection) { - if (connection == null) { - issues.error("Cannot run pre-delete dependency check without a connection"); - return; - } try { PipelineDependencyChecker.assertNoExternalDependents( K8sContext.create(connection), source.database(), source.path(), selfOwnerUid); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java index bd70ae22..0a9cfedf 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java @@ -24,7 +24,7 @@ *

{@link K8sApi#update} merges labels additively, so stale {@code depends-on-*} labels from * a previous version of the pipeline's SQL can linger. Correctness is preserved by the * annotation, which is rewritten in full on every update: the checker rejects any label-only - * match whose annotation doesn't list the target identifier. In return we avoid the extra + * match whose annotation doesn't list the target identifier. In return, we avoid the extra * round trip that in-place label stripping would require. */ class K8sPipelineDeployer extends K8sDeployer { @@ -35,10 +35,6 @@ class K8sPipelineDeployer extends K8sDeployer sources; private final Sink sink; - K8sPipelineDeployer(String name, List specs, String sql, K8sContext context) { - this(name, specs, sql, Collections.emptyList(), null, context); - } - K8sPipelineDeployer(String name, List specs, String sql, Collection sources, Sink sink, K8sContext context) { super(context, K8sApiEndpoints.PIPELINES); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java index 7d3d955f..be14566a 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java @@ -16,8 +16,6 @@ * {@code PendingDelete} (not raw {@code Source}) makes the guard explicitly delete-time: * other callers of {@code ValidationService.validateOrThrow(source, connection)} won't trigger * a pre-delete lookup against K8s. - * - *

Registered via {@code META-INF/services/com.linkedin.hoptimator.ValidatorProvider}. */ public class K8sValidatorProvider implements ValidatorProvider { diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java index 122f817d..5ea20197 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java @@ -12,6 +12,8 @@ import com.linkedin.hoptimator.k8s.models.V1alpha1Pipeline; import com.linkedin.hoptimator.k8s.models.V1alpha1PipelineList; +import javax.annotation.Nullable; + /** * Checks whether any Pipeline CRDs still depend on a resource a {@link com.linkedin.hoptimator.Deployer} @@ -32,14 +34,14 @@ private PipelineDependencyChecker() { } public static void assertNoExternalDependents(K8sContext context, String database, - List path, String selfOwnerUid) throws SQLException { + List path, @Nullable String selfOwnerUid) throws SQLException { assertNoExternalDependents(new K8sApi<>(context, K8sApiEndpoints.PIPELINES), database, path, selfOwnerUid); } /** Variant that takes a pre-built {@link K8sApi} — used by tests to inject mocks. */ static void assertNoExternalDependents(K8sApi api, - String database, List path, String selfOwnerUid) throws SQLException { + String database, List path, @Nullable String selfOwnerUid) throws SQLException { String labelKey = PipelineDependencyLabels.labelKey(database, path); String identifier = PipelineDependencyLabels.identifier(database, path); @@ -65,7 +67,7 @@ static void assertNoExternalDependents(K8sApi Date: Fri, 1 May 2026 14:30:10 -0400 Subject: [PATCH 06/15] Migrate UID lookup to kind+name --- .../linkedin/hoptimator/PendingDelete.java | 32 ++++++--- .../k8s/K8sPipelineDependencyValidator.java | 15 ++-- .../hoptimator/k8s/K8sValidatorProvider.java | 2 +- .../k8s/PipelineDependencyChecker.java | 23 +++--- .../k8s/PipelineDependencyCheckerTest.java | 44 ++++++------ .../logical/LogicalTableDeployer.java | 45 +++--------- .../logical/LogicalTableDeployerTest.java | 70 +++++-------------- 7 files changed, 96 insertions(+), 135 deletions(-) diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java index 3c93f993..5c330aa8 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/PendingDelete.java @@ -11,35 +11,45 @@ * caller of {@code ValidationService.validateOrThrow(source, connection)} doesn't accidentally * trigger delete-intent checks. * - *

An optional {@code selfOwnerUid} lets the caller declare an "umbrella" K8s resource UID - * whose owned objects should be excluded from the dependent set — e.g. a LogicalTable CRD's UID, - * so its child Pipeline CRDs (which reference tier sources by SQL) don't self-block the drop. + *

An optional {@code (selfOwnerKind, selfOwnerName)} lets the caller declare an "umbrella" + * K8s resource whose owned objects should be excluded from the dependent set — e.g. a + * LogicalTable CRD, so its child Pipeline CRDs (which reference tier sources by SQL) don't + * self-block the drop. */ public final class PendingDelete { private final T target; - private final String selfOwnerUid; + private final String selfOwnerKind; + private final String selfOwnerName; public PendingDelete(T target) { - this(target, null); + this(target, null, null); } - public PendingDelete(T target, @Nullable String selfOwnerUid) { + public PendingDelete(T target, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) { this.target = Objects.requireNonNull(target, "target"); - this.selfOwnerUid = selfOwnerUid; + this.selfOwnerKind = selfOwnerKind; + this.selfOwnerName = selfOwnerName; } public T target() { return target; } - /** UID of the K8s resource whose owned objects should be excluded from the dependent set. */ - public @Nullable String selfOwnerUid() { - return selfOwnerUid; + /** Kind of the K8s resource whose owned objects should be excluded from the dependent set. */ + public @Nullable String selfOwnerKind() { + return selfOwnerKind; + } + + /** Name of the K8s resource whose owned objects should be excluded from the dependent set. */ + public @Nullable String selfOwnerName() { + return selfOwnerName; } @Override public String toString() { - return "PendingDelete[" + target + (selfOwnerUid != null ? ", self=" + selfOwnerUid : "") + "]"; + String self = (selfOwnerKind != null && selfOwnerName != null) + ? ", self=" + selfOwnerKind + "/" + selfOwnerName : ""; + return "PendingDelete[" + target + self + "]"; } } diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java index 5c17e2ee..ff06a3be 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java @@ -11,24 +11,27 @@ /** * Pre-delete dependency check, run by the validator framework when a {@link Source} is wrapped * in {@link com.linkedin.hoptimator.PendingDelete}. Delegates to the existing - * {@link PipelineDependencyChecker} (label-selector + annotation collision-guard + self-uid - * exclusion) and surfaces any blocking pipeline as a validation error + * {@link PipelineDependencyChecker} (label-selector + annotation collision-guard + self-owner + * exclusion) and surfaces any blocking pipeline as a validation error. */ final class K8sPipelineDependencyValidator implements Validator { private final Source source; - private final String selfOwnerUid; + private final String selfOwnerKind; + private final String selfOwnerName; - K8sPipelineDependencyValidator(Source source, @Nullable String selfOwnerUid) { + K8sPipelineDependencyValidator(Source source, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) { this.source = source; - this.selfOwnerUid = selfOwnerUid; + this.selfOwnerKind = selfOwnerKind; + this.selfOwnerName = selfOwnerName; } @Override public void validate(Issues issues, Connection connection) { try { PipelineDependencyChecker.assertNoExternalDependents( - K8sContext.create(connection), source.database(), source.path(), selfOwnerUid); + K8sContext.create(connection), source.database(), source.path(), + selfOwnerKind, selfOwnerName); } catch (SQLException e) { issues.error(e.getMessage()); } diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java index be14566a..3cc45e4b 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java @@ -26,7 +26,7 @@ public Collection validators(T obj, Connection connection) { Object target = pd.target(); if (target instanceof Source) { return Collections.singletonList( - new K8sPipelineDependencyValidator((Source) target, pd.selfOwnerUid())); + new K8sPipelineDependencyValidator((Source) target, pd.selfOwnerKind(), pd.selfOwnerName())); } } return Collections.emptyList(); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java index 5ea20197..435dee5e 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java @@ -24,9 +24,10 @@ * {@link PipelineDependencyLabels#ANNOTATION_KEY} annotation to rule out the (rare) case of a * hash collision in the label slug. * - *

Pipelines owned (directly) by {@code selfOwnerUid} are excluded from the blocker list: those - * pipelines will be cascade-deleted alongside the parent resource, so counting them as external - * dependents would make composite deletes (e.g. {@code LogicalTableDeployer.delete()}) impossible. + *

Pipelines owned (directly) by {@code (selfOwnerKind, selfOwnerName)} are excluded from the + * blocker list: those pipelines will be cascade-deleted alongside the parent resource, so counting + * them as external dependents would make composite deletes (e.g. {@code LogicalTableDeployer.delete()}) + * impossible. */ public final class PipelineDependencyChecker { @@ -34,14 +35,15 @@ private PipelineDependencyChecker() { } public static void assertNoExternalDependents(K8sContext context, String database, - List path, @Nullable String selfOwnerUid) throws SQLException { + List path, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) throws SQLException { assertNoExternalDependents(new K8sApi<>(context, K8sApiEndpoints.PIPELINES), - database, path, selfOwnerUid); + database, path, selfOwnerKind, selfOwnerName); } /** Variant that takes a pre-built {@link K8sApi} — used by tests to inject mocks. */ static void assertNoExternalDependents(K8sApi api, - String database, List path, @Nullable String selfOwnerUid) throws SQLException { + String database, List path, @Nullable String selfOwnerKind, + @Nullable String selfOwnerName) throws SQLException { String labelKey = PipelineDependencyLabels.labelKey(database, path); String identifier = PipelineDependencyLabels.identifier(database, path); @@ -50,7 +52,7 @@ static void assertNoExternalDependents(K8sApi blockers = new ArrayList<>(); for (V1alpha1Pipeline p : matches) { - if (isSelfOwned(p, selfOwnerUid)) { + if (isSelfOwned(p, selfOwnerKind, selfOwnerName)) { continue; } if (!annotationConfirms(p, identifier)) { @@ -67,8 +69,9 @@ static void assertNoExternalDependents(K8sApi PATH = Collections.singletonList("my-topic"); private static final String IDENTIFIER = "kafka1_my-topic"; - private static V1alpha1Pipeline pipeline(String name, String ownerUid, String annotationValue) { + private static V1alpha1Pipeline pipeline(String name, String ownerKind, String ownerName, + String annotationValue) { V1ObjectMeta meta = new V1ObjectMeta().name(name); - if (ownerUid != null) { - meta.addOwnerReferencesItem(new V1OwnerReference().kind("View").name("owner").uid(ownerUid)); + if (ownerKind != null && ownerName != null) { + meta.addOwnerReferencesItem(new V1OwnerReference().kind(ownerKind).name(ownerName)); } if (annotationValue != null) { Map annotations = new HashMap<>(); @@ -51,16 +52,16 @@ private static V1alpha1Pipeline pipeline(String name, String ownerUid, String an void passesWhenNoPipelinesMatch() throws SQLException { when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Collections.emptyList()); - assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); } @Test void blocksOnExternalPipeline() throws SQLException { when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) - .thenReturn(Collections.singletonList(pipeline("ext-pipe", "other-uid", IDENTIFIER))); + .thenReturn(Collections.singletonList(pipeline("ext-pipe", "View", "owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("ext-pipe")); assertTrue(ex.getMessage().contains(IDENTIFIER)); } @@ -68,20 +69,21 @@ void blocksOnExternalPipeline() throws SQLException { @Test void skipsSelfOwnedPipeline() throws SQLException { when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) - .thenReturn(Collections.singletonList(pipeline("owned-pipe", "self-uid", IDENTIFIER))); + .thenReturn(Collections.singletonList(pipeline("owned-pipe", "LogicalTable", "self-name", IDENTIFIER))); assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( - api, DB, PATH, "self-uid")); + api, DB, PATH, "LogicalTable", "self-name")); } @Test void blocksOnExternalWhenSomeAreSelfOwned() throws SQLException { when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( - pipeline("owned-pipe", "self-uid", IDENTIFIER), - pipeline("external-pipe", "other-uid", IDENTIFIER))); + pipeline("owned-pipe", "LogicalTable", "self-name", IDENTIFIER), + pipeline("external-pipe", "View", "other-owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, "self-uid")); + () -> PipelineDependencyChecker.assertNoExternalDependents( + api, DB, PATH, "LogicalTable", "self-name")); assertTrue(ex.getMessage().contains("external-pipe")); assertTrue(!ex.getMessage().contains("owned-pipe"), "self-owned pipeline must not be listed"); } @@ -91,10 +93,10 @@ void rejectsSlugCollisionViaAnnotation() throws SQLException { // Pipeline labels collide on the slug (which is what api.select matched on) but the // annotation reveals the actual identifier is different — so this should NOT block. when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) - .thenReturn(Collections.singletonList(pipeline("colliding-pipe", "other-uid", + .thenReturn(Collections.singletonList(pipeline("colliding-pipe", "View", "owner", "some-other-database/some-other-path"))); - assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); } @Test @@ -102,20 +104,20 @@ void treatsMissingAnnotationAsTrusted() throws SQLException { // A pipeline with the matching label but no depends-on annotation (pre-labeling migration // case, or future code path that didn't write the annotation) is still treated as a blocker. when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) - .thenReturn(Collections.singletonList(pipeline("legacy-pipe", "other-uid", null))); + .thenReturn(Collections.singletonList(pipeline("legacy-pipe", "View", "owner", null))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("legacy-pipe")); } @Test void errorMessageIncludesOwnerKindAndName() throws SQLException { when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) - .thenReturn(Collections.singletonList(pipeline("ext-pipe", "other-uid", IDENTIFIER))); + .thenReturn(Collections.singletonList(pipeline("ext-pipe", "View", "owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("View/owner"), "error should name the owning View so the user knows what to unhook: " + ex.getMessage()); } @@ -123,12 +125,12 @@ void errorMessageIncludesOwnerKindAndName() throws SQLException { @Test void errorMessageListsAllBlockers() throws SQLException { when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( - pipeline("p1", "uid1", IDENTIFIER), - pipeline("p2", "uid2", IDENTIFIER), - pipeline("p3", "uid3", IDENTIFIER))); + pipeline("p1", "View", "owner1", IDENTIFIER), + pipeline("p2", "View", "owner2", IDENTIFIER), + pipeline("p3", "View", "owner3", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null)); + () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("p1")); assertTrue(ex.getMessage().contains("p2")); assertTrue(ex.getMessage().contains("p3")); diff --git a/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java b/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java index b2daf305..dbaf7ba3 100644 --- a/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java +++ b/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java @@ -16,8 +16,6 @@ import com.linkedin.hoptimator.k8s.models.V1alpha1DatabaseSpec; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplate; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplateList; -import com.linkedin.hoptimator.k8s.models.V1alpha1LogicalTable; -import com.linkedin.hoptimator.k8s.models.V1alpha1LogicalTableList; import com.linkedin.hoptimator.k8s.models.V1alpha1TableTrigger; import com.linkedin.hoptimator.k8s.models.V1alpha1TableTriggerList; import com.linkedin.hoptimator.util.planner.PipelineRel; @@ -71,7 +69,6 @@ public class LogicalTableDeployer implements Deployer, Validated { private final Properties tierProps; private final K8sContext context; private final K8sApi databasesApi; - private final K8sApi logicalTableApi; private final List tierDeployers = new ArrayList<>(); private final List pipelineDeployers = new ArrayList<>(); @@ -88,20 +85,16 @@ public class LogicalTableDeployer implements Deployer, Validated { private Map cachedTierSources; LogicalTableDeployer(Source source, Properties tierProps, K8sContext context) { - this(source, tierProps, context, - new K8sApi<>(context, K8sApiEndpoints.DATABASES), - new K8sApi<>(context, K8sApiEndpoints.LOGICAL_TABLES)); + this(source, tierProps, context, new K8sApi<>(context, K8sApiEndpoints.DATABASES)); } - /** Package-private constructor for testing — accepts injectable K8s APIs. */ + /** Package-private constructor for testing — accepts an injectable Database K8s API. */ LogicalTableDeployer(Source source, Properties tierProps, K8sContext context, - K8sApi databasesApi, - K8sApi logicalTableApi) { + K8sApi databasesApi) { this.source = source; this.tierProps = tierProps; this.context = context; this.databasesApi = databasesApi; - this.logicalTableApi = logicalTableApi; } /** @@ -215,33 +208,15 @@ private void deployAll(boolean update) throws SQLException { } } - /** - * Looks up the existing LogicalTable CRD's UID, used as the {@code selfOwnerUid} on - * {@link PendingDelete} so the pre-delete dep check excludes pipelines owned by this CRD - * (the implicit inter-tier pipelines are cascade-deleted with it). Returns {@code null} if - * the CRD doesn't exist (pre-create or already-deleted state). - */ - private String existingLogicalTableUid() throws SQLException { - if (logicalTableApi == null) { - return null; - } - String crdName = K8sUtils.canonicalizeName(source.path()); - V1alpha1LogicalTable existing = logicalTableApi.getIfExists(context.namespace(), crdName); - if (existing == null || existing.getMetadata() == null) { - return null; - } - return existing.getMetadata().getUid(); - } - /** * Deletes a logical table. * *

A logical DROP is structurally equivalent to running DROP TABLE on each tier plus * deleting the LogicalTable CRD. We mirror that shape exactly: each tier goes through the * same {@code validateOrThrow → DeploymentService.delete} pipeline a standalone DROP would. - * The {@link PendingDelete}'s {@code selfOwnerUid} is the LogicalTable CRD's UID so that the - * implicit inter-tier pipelines (owned by the CRD, cascade-deleted with it) are excluded - * from the dependent set — only external pipelines block. + * The {@link PendingDelete}'s {@code (selfOwnerKind, selfOwnerName)} pair identifies the + * LogicalTable CRD so the implicit inter-tier pipelines (owned by the CRD, cascade-deleted + * with it) are excluded from the dependent set — only external pipelines block. * *

    *
  1. Per-tier dep check via the validator framework. Any active external pipeline blocks. @@ -256,16 +231,16 @@ private String existingLogicalTableUid() throws SQLException { public void delete() throws SQLException { Map tierSources = buildTierSources(); HoptimatorConnection conn = context.connection(); - String selfUid = existingLogicalTableUid(); + String selfName = K8sUtils.canonicalizeName(source.path()); // 1. Per-tier pre-flight dep check. for (Source tierSource : tierSources.values()) { - ValidationService.validateOrThrow(new PendingDelete<>(tierSource, selfUid), conn); + ValidationService.validateOrThrow( + new PendingDelete<>(tierSource, "LogicalTable", selfName), conn); } // 2. Delete the LogicalTable CRD (cascades owned pipelines/triggers). - createLogicalTableDeployer( - K8sUtils.canonicalizeName(source.path()), source.database(), buildTierMap()).delete(); + createLogicalTableDeployer(selfName, source.database(), buildTierMap()).delete(); // 3. Per-tier physical cleanup. Best-effort: only deregister a tier's schema entry when its // physical delete succeeded; failed tiers keep their entries so the user can retry. diff --git a/hoptimator-logical/src/test/java/com/linkedin/hoptimator/logical/LogicalTableDeployerTest.java b/hoptimator-logical/src/test/java/com/linkedin/hoptimator/logical/LogicalTableDeployerTest.java index cab193a8..a2d4015a 100644 --- a/hoptimator-logical/src/test/java/com/linkedin/hoptimator/logical/LogicalTableDeployerTest.java +++ b/hoptimator-logical/src/test/java/com/linkedin/hoptimator/logical/LogicalTableDeployerTest.java @@ -40,8 +40,6 @@ import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplate; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplateList; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplateSpec; -import com.linkedin.hoptimator.k8s.models.V1alpha1LogicalTable; -import com.linkedin.hoptimator.k8s.models.V1alpha1LogicalTableList; import com.linkedin.hoptimator.k8s.models.V1alpha1LogicalTableSpecTiers; import com.linkedin.hoptimator.k8s.models.V1alpha1TableTrigger; import com.linkedin.hoptimator.k8s.models.V1alpha1TableTriggerList; @@ -139,9 +137,7 @@ private static Source testSource() { private LogicalTableDeployer deployerWithMockCrd( Source src, Properties props, K8sContext ctx, FakeK8sApi dbApi) { - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); - return new LogicalTableDeployer(src, props, ctx, dbApi, ltApi) { + return new LogicalTableDeployer(src, props, ctx, dbApi) { @Override K8sLogicalTableDeployer createLogicalTableDeployer( String crdName, String databaseLabel, Map tierMap) { @@ -245,13 +241,10 @@ void pipelineNameNearlineToOnline() { // delete() / DependencyGuarded / specify() tests - /** Builds a 2-tier deployer with mocked CRD deployer and pre-populated fake K8s APIs. */ - private LogicalTableDeployer deployerWithApis(Properties props, - List dbs, List logicalTables) { + /** Builds a 2-tier deployer with mocked CRD deployer and a pre-populated fake Database API. */ + private LogicalTableDeployer deployerWithApis(Properties props, List dbs) { FakeK8sApi dbApi = new FakeK8sApi<>(new ArrayList<>(dbs)); - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>(logicalTables)); - return new LogicalTableDeployer(testSource(), props, mockContext(), dbApi, ltApi) { + return new LogicalTableDeployer(testSource(), props, mockContext(), dbApi) { @Override K8sLogicalTableDeployer createLogicalTableDeployer( String crdName, String databaseLabel, Map tierMap) { @@ -264,8 +257,7 @@ K8sLogicalTableDeployer createLogicalTableDeployer( void deleteThrowsWhenCrdDeleteFails() throws SQLException { LogicalTableDeployer deployer = deployerWithApis( twoTierProps("kafka-db", "venice-db"), - Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), - Collections.emptyList()); + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE"))); doThrow(new SQLException("crd gone")).when(mockCrdDeployer).delete(); @@ -281,8 +273,7 @@ void deleteThrowsWhenCrdDeleteFails() throws SQLException { void deleteSwallowsTierDeleteFailuresAndContinues() throws SQLException { LogicalTableDeployer deployer = deployerWithApis( twoTierProps("kafka-db", "venice-db"), - Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), - Collections.emptyList()); + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE"))); // Two tier deployers: first throws, second succeeds. The overall delete must return cleanly. Deployer failing = mock(Deployer.class); @@ -320,8 +311,7 @@ private void withMockedDdlUtils(Runnable body, Consumer DeploymentService.deployers(any(), any())) @@ -349,8 +339,7 @@ void deleteKeepsSchemaEntryForTierWhoseDeleteFailed() throws SQLException { // The failed tier's schema entry must NOT be removed; the succeeded tier's must be. LogicalTableDeployer deployer = deployerWithApis( twoTierProps("kafka-db", "venice-db"), - Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), - Collections.emptyList()); + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE"))); Deployer failingTier = mock(Deployer.class); Deployer succeedingTier = mock(Deployer.class); @@ -381,8 +370,7 @@ void deleteKeepsSchemaEntryForTierWhoseDeleteFailed() throws SQLException { void deleteRunsCrdDeleteBeforeTierDeletes() throws SQLException { LogicalTableDeployer deployer = deployerWithApis( twoTierProps("kafka-db", "venice-db"), - Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE")), - Collections.emptyList()); + Arrays.asList(makeDb("kafka-db", "KAFKA"), makeDb("venice-db", "VENICE"))); Deployer tierDeployer = mock(Deployer.class); deploymentServiceMock.when(() -> DeploymentService.deployers(any(), any())) @@ -644,9 +632,7 @@ void createWithNearlineAndOnlineTiersAttemptsPipelineDeployment() throws Excepti // Use a subclass that mocks the CRD deployer but does NOT suppress deployPipelineBundle, // so the pipeline path is exercised and fails due to the null connection in mockContext(). - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); - LogicalTableDeployer deployer = new LogicalTableDeployer(testSource(), props, mockContext(), dbApi, ltApi) { + LogicalTableDeployer deployer = new LogicalTableDeployer(testSource(), props, mockContext(), dbApi) { @Override K8sLogicalTableDeployer createLogicalTableDeployer( String crdName, String databaseLabel, Map tierMap) { @@ -669,11 +655,9 @@ void validateSucceedsWithValidTierConfiguration() throws Exception { FakeK8sApi dbApi = new FakeK8sApi<>(Arrays.asList(makeDb("nearline-db", "NEARLINE"), makeDb("offline-db", "OFFLINE"))); - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); Validator.Issues issues = new Validator.Issues("test"); new LogicalTableDeployer( - testSource(), twoTierProps("nearline-db", "offline-db"), mockContext(), dbApi, ltApi) + testSource(), twoTierProps("nearline-db", "offline-db"), mockContext(), dbApi) .validate(issues, null); assertTrue(issues.valid()); @@ -681,12 +665,10 @@ void validateSucceedsWithValidTierConfiguration() throws Exception { @Test void validateReportsIssueWhenDatabaseNotFound() throws Exception { - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); Validator.Issues issues = new Validator.Issues("test"); new LogicalTableDeployer( testSource(), twoTierProps("missing-db", "also-missing"), - mockContext(), new FakeK8sApi<>(new ArrayList<>()), ltApi) + mockContext(), new FakeK8sApi<>(new ArrayList<>())) .validate(issues, null); assertFalse(issues.valid()); @@ -704,12 +686,10 @@ void validateCallsValidatedDeployersWhenTiersExist() throws Exception { new FakeK8sApi<>(Arrays.asList(makeDb("nearline-db", "NEARLINE"))); K8sContext ctx = mock(K8sContext.class); - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); Validator.Issues issues = new Validator.Issues("test"); new LogicalTableDeployer( - makeSource("logical", "testevent"), oneTierProps, ctx, dbApi, ltApi) + makeSource("logical", "testevent"), oneTierProps, ctx, dbApi) .validate(issues, null); verify(mockValidatedDeployer).validate(issues, null); @@ -753,11 +733,9 @@ void ensureTierRowTypesRegisteredWithConnectionRecordsRowTypeError() throws Exce FakeK8sApi dbApi = new FakeK8sApi<>(Arrays.asList(makeDb("nearline-db", "NEARLINE"))); - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); Validator.Issues issues = new Validator.Issues("test"); new LogicalTableDeployer( - makeSource("logical", "testevent"), oneTierProps, ctx, dbApi, ltApi) + makeSource("logical", "testevent"), oneTierProps, ctx, dbApi) .validate(issues, null); assertFalse(issues.valid()); @@ -773,9 +751,7 @@ void specifyWithNearlineAndOnlineThrowsException() { Properties props = twoTierProps("nearline-db", "online-db"); props.setProperty(LogicalTier.ONLINE.tierName(), "online-db"); - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); - assertThrows(Exception.class, () -> new LogicalTableDeployer(testSource(), props, mockContext(), dbApi, ltApi).specify()); + assertThrows(Exception.class, () -> new LogicalTableDeployer(testSource(), props, mockContext(), dbApi).specify()); } @Test @@ -786,10 +762,8 @@ void specifyWithOfflineTierOnlyDoesNotAttemptPipeline() throws Exception { Properties props = new Properties(); props.setProperty(LogicalTier.OFFLINE.tierName(), "offline-db"); - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); List specs = new LogicalTableDeployer( - testSource(), props, mockContext(), dbApi, ltApi).specify(); + testSource(), props, mockContext(), dbApi).specify(); assertNotNull(specs); assertTrue(specs.isEmpty(), "offline-only — no pipeline spec should be attempted"); @@ -810,11 +784,9 @@ void specifyIncludesTierResourceSpecsFromDeploymentService() throws Exception { // specify() calls DeploymentService.specify() per tier before the pipeline path, // which fails (null connection) — so we only see tier specs, not job specs. - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); List specs; try { - specs = new LogicalTableDeployer(testSource(), props, mockContext(), dbApi, ltApi).specify(); + specs = new LogicalTableDeployer(testSource(), props, mockContext(), dbApi).specify(); } catch (SQLException ignored) { // Pipeline planning may throw due to null connection; tier specs are added first. return; @@ -829,11 +801,9 @@ void specifyWithNearlineAndOfflineThrowsExceptionOnPipelinePlanning() { FakeK8sApi dbApi = new FakeK8sApi<>(Arrays.asList(makeDb("nearline-db", "NEARLINE"), makeDb("offline-db", "OFFLINE"))); - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); assertThrows(Exception.class, () -> new LogicalTableDeployer(testSource(), twoTierProps("nearline-db", "offline-db"), - mockContext(), dbApi, ltApi).specify()); + mockContext(), dbApi).specify()); } @Test @@ -877,9 +847,7 @@ private LogicalTableDeployer deployerWithJobTemplates( new FakeK8sApi<>(jobTemplates); FakeK8sApi triggerApi = new FakeK8sApi<>(preExistingTriggers); - FakeK8sApi ltApi = - new FakeK8sApi<>(new ArrayList<>()); - return new LogicalTableDeployer(src, props, ctx, dbApi, ltApi) { + return new LogicalTableDeployer(src, props, ctx, dbApi) { @Override K8sLogicalTableDeployer createLogicalTableDeployer( String crdName, String databaseLabel, Map tierMap) { From fa1a5d6a2c84490c0256c85e7e799d5846afb356 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Fri, 1 May 2026 15:22:59 -0400 Subject: [PATCH 07/15] Fix issue to preserve existing annotations --- .../com/linkedin/hoptimator/k8s/K8sApi.java | 11 ++- .../linkedin/hoptimator/k8s/K8sYamlApi.java | 11 ++- .../k8s/PipelineDependencyChecker.java | 2 +- .../k8s/PipelineDependencyLabels.java | 34 ++------- .../linkedin/hoptimator/k8s/K8sApiTest.java | 52 ++++++++++++++ .../hoptimator/k8s/K8sYamlApiTest.java | 69 +++++++++++++++++++ .../k8s/PipelineDependencyCheckerTest.java | 3 +- .../k8s/PipelineDependencyLabelsTest.java | 22 ------ .../logical/LogicalTableDeployerTest.java | 2 - .../hoptimator/venice/ClusterSchemaTest.java | 2 +- 10 files changed, 152 insertions(+), 56 deletions(-) diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sApi.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sApi.java index 0f19919e..23ee79ef 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sApi.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sApi.java @@ -154,7 +154,7 @@ public void update(T obj) throws SQLException { final KubernetesApiResponse resp; if (existing.isSuccess()) { - // Ensure labels are additive. + // Ensure labels, annotations, and owners are additive. Map labels = new HashMap<>(); if (existing.getObject().getMetadata().getLabels() != null) { labels.putAll(existing.getObject().getMetadata().getLabels()); @@ -164,6 +164,15 @@ public void update(T obj) throws SQLException { } obj.getMetadata().setLabels(labels); + Map annotations = new HashMap<>(); + if (existing.getObject().getMetadata().getAnnotations() != null) { + annotations.putAll(existing.getObject().getMetadata().getAnnotations()); + } + if (obj.getMetadata().getAnnotations() != null) { + annotations.putAll(obj.getMetadata().getAnnotations()); + } + obj.getMetadata().setAnnotations(annotations); + List owners = new LinkedList<>(); if (existing.getObject().getMetadata().getOwnerReferences() != null) { owners.addAll(existing.getObject().getMetadata().getOwnerReferences()); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sYamlApi.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sYamlApi.java index bc14dc6d..76fa7857 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sYamlApi.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sYamlApi.java @@ -115,7 +115,7 @@ public void update(DynamicKubernetesObject obj) throws SQLException { final KubernetesApiResponse resp; if (existing.isSuccess()) { - // Ensure labels are additive. Existing values are kept. + // Ensure labels, annotations are additive. Existing values are kept. Map labels = new HashMap<>(); if (obj.getMetadata().getLabels() != null) { labels.putAll(obj.getMetadata().getLabels()); @@ -125,6 +125,15 @@ public void update(DynamicKubernetesObject obj) throws SQLException { } existing.getObject().getMetadata().setLabels(labels); + Map annotations = new HashMap<>(); + if (obj.getMetadata().getAnnotations() != null) { + annotations.putAll(obj.getMetadata().getAnnotations()); + } + if (existing.getObject().getMetadata().getAnnotations() != null) { + annotations.putAll(existing.getObject().getMetadata().getAnnotations()); + } + existing.getObject().getMetadata().setAnnotations(annotations); + obj.setMetadata(existing.getObject().getMetadata()); resp = context.dynamic(obj.getApiVersion(), K8sUtils.guessPlural(obj)).update(obj); } else { diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java index 435dee5e..ee9a6b64 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java @@ -56,7 +56,7 @@ static void assertNoExternalDependents(K8sApiEvery source and sink a pipeline references is recorded as a label: - * {@code hoptimator.linkedin.com/depends-on-: "/"} where - * {@code } is a deterministic hash derived from {@code database + "/" + pathString}. + * {@code hoptimator.linkedin.com/depends-on-: "_"} where + * {@code } is a deterministic hash derived from {@code database + "_" + pathString}. * The hash keeps label keys within Kubernetes's 63-character name limit for arbitrary paths, * and lets {@code K8sApi.select} filter pipelines by dependency on the server. * @@ -39,9 +39,6 @@ private PipelineDependencyLabels() { /** * Canonical logical identifier for a resource: {@code _}. - * - *

    The separator is {@code _} (not {@code /}) so the result is also a valid Kubernetes - * label value out of the box — K8s allows {@code [A-Za-z0-9_.-]} but not {@code /}. */ public static String identifier(String database, Iterable path) { return database + "_" + String.join(".", path); @@ -65,13 +62,11 @@ public static String labelKey(String database, Iterable path) { /** * Labels to stamp on a Pipeline CRD — one entry per source and the sink. Both edges * matter to the guard: dropping a source orphans pipelines that read from it; dropping a sink - * orphans pipelines that write to it. The partial-view scenario where two pipelines share a - * sink (e.g. {@code X} and {@code X$piece}) is unaffected — DROP MV routes through - * {@code K8sViewDeployer}, which deliberately does not implement {@code DependencyGuarded} - * (DROP MV is metadata-only and does not destroy the underlying physical sink). + * orphans pipelines that write to it. * *

    Keys are the same as {@link #labelKey}. Values are the readable identifier, truncated - * to 63 chars if necessary (the annotation preserves the untruncated form). + * to 63 chars if necessary (the annotation preserves the untruncated form). Values are + * for debugging purposes only. */ public static Map labelsFor(Collection sources, Sink sink) { Map labels = new LinkedHashMap<>(); @@ -86,8 +81,8 @@ public static Map labelsFor(Collection sources, Sink sin /** * Collision-guard annotation value — comma-separated list of full source and sink identifiers, - * deduplicated. The delete-time check cross-references this annotation after the label - * selector narrows the candidate set. + * deduplicated and not truncated. The delete-time check cross-references this annotation after + * the label selector narrows the candidate set. */ public static String annotationFor(Collection sources, Sink sink) { Set ids = new LinkedHashSet<>(); @@ -100,21 +95,6 @@ public static String annotationFor(Collection sources, Sink sink) { return String.join(",", ids); } - /** - * Removes any {@code depends-on-*} entries from an existing label map so that an update can - * restamp the current dependency set without carrying stale labels from an earlier version of - * the pipeline. {@link K8sApi#update} is additive for labels; callers must strip first. - */ - public static Map stripDependencyLabels(Map labels) { - if (labels == null) { - return new LinkedHashMap<>(); - } - return labels.entrySet().stream() - .filter(e -> !e.getKey().startsWith(LABEL_PREFIX)) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, - (a, b) -> a, LinkedHashMap::new)); - } - /** Parses the collision-guard annotation back into the set of identifiers it encoded. */ public static Set parseAnnotation(String annotation) { Set out = new LinkedHashSet<>(); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sApiTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sApiTest.java index 92414370..f1e52c26 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sApiTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sApiTest.java @@ -276,6 +276,58 @@ void updateMergesLabelsFromExisting() throws SQLException { assertEquals("new-val", mergedLabels.get("new-key")); } + @Test + void updateMergesAnnotationsFromExisting() throws SQLException { + V1alpha1Pipeline pipeline = makePipeline("existing", "test-ns"); + Map newAnnotations = new HashMap<>(); + newAnnotations.put("new-key", "new-val"); + pipeline.getMetadata().setAnnotations(newAnnotations); + + V1alpha1Pipeline existing = makePipeline("existing", "test-ns"); + Map existingAnnotations = new HashMap<>(); + existingAnnotations.put("existing-key", "existing-val"); + existing.getMetadata().setAnnotations(existingAnnotations); + existing.getMetadata().setResourceVersion("rv2"); + + when(mockGenericApi.get(eq("test-ns"), eq("existing"))).thenReturn(mockSingleResponse); + when(mockSingleResponse.isSuccess()).thenReturn(true); + when(mockSingleResponse.getObject()).thenReturn(existing); + when(mockGenericApi.update(any(V1alpha1Pipeline.class))).thenReturn(mockSingleResponse); + + api.update(pipeline); + + Map mergedAnnotations = pipeline.getMetadata().getAnnotations(); + assertEquals("existing-val", mergedAnnotations.get("existing-key")); + assertEquals("new-val", mergedAnnotations.get("new-key")); + } + + @Test + void updateLocalAnnotationWinsOnSharedKey() throws SQLException { + // Locks in the freshness guarantee the dependency-guard relies on: when the local object + // sets the same annotation key the cluster's existing object had, the local value wins. + // Without this, the depends-on annotation would never refresh on CREATE OR REPLACE and stale + // labels could no longer be disambiguated by the collision-guard. + V1alpha1Pipeline pipeline = makePipeline("existing", "test-ns"); + Map newAnnotations = new HashMap<>(); + newAnnotations.put("shared-key", "fresh-val"); + pipeline.getMetadata().setAnnotations(newAnnotations); + + V1alpha1Pipeline existing = makePipeline("existing", "test-ns"); + Map existingAnnotations = new HashMap<>(); + existingAnnotations.put("shared-key", "stale-val"); + existing.getMetadata().setAnnotations(existingAnnotations); + existing.getMetadata().setResourceVersion("rv2"); + + when(mockGenericApi.get(eq("test-ns"), eq("existing"))).thenReturn(mockSingleResponse); + when(mockSingleResponse.isSuccess()).thenReturn(true); + when(mockSingleResponse.getObject()).thenReturn(existing); + when(mockGenericApi.update(any(V1alpha1Pipeline.class))).thenReturn(mockSingleResponse); + + api.update(pipeline); + + assertEquals("fresh-val", pipeline.getMetadata().getAnnotations().get("shared-key")); + } + @Test void updateWhenObjectNotExistsCallsCreate() throws SQLException { V1alpha1Pipeline pipeline = makePipeline("new-pipeline", "test-ns"); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sYamlApiTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sYamlApiTest.java index 8f8a623a..d640d7da 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sYamlApiTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sYamlApiTest.java @@ -849,6 +849,75 @@ void updateExistingObjectWithNewLabelsOnlyPreservesAll() throws SQLException { assertNotNull(capturedLabels); assertTrue(capturedLabels.containsKey("existing"), "Should contain existing label"); } + + @Test + void updateExistingPreservesExistingAnnotationsOverNewOnes() throws SQLException { + // obj.setMetadata(existing.getObject().getMetadata()) — existing metadata replaces obj's. + // After update, existing labels are present on the object passed to dynApi.update(). + DynamicKubernetesObject existingObj = new DynamicKubernetesObject(); + existingObj.setApiVersion("v1"); + existingObj.setKind("ConfigMap"); + Map existingAnnotations = new HashMap<>(); + existingAnnotations.put("key", "existing-value"); + existingObj.setMetadata(new V1ObjectMeta().name("target").namespace("ns").annotations(existingAnnotations)); + + DynamicKubernetesApi dynApi = mock(DynamicKubernetesApi.class); + doReturn(dynApi).when(mockContext).dynamic(anyString(), anyString()); + doReturn(successResponse(existingObj)).when(dynApi).get(anyString(), anyString()); + doReturn(successResponse(existingObj)).when(dynApi).update(any(DynamicKubernetesObject.class)); + + K8sYamlApi api = new K8sYamlApi(mockContext); + DynamicKubernetesObject obj = new DynamicKubernetesObject(); + obj.setApiVersion("v1"); + obj.setKind("ConfigMap"); + // obj starts with different annotation value for same key + Map newAnnotations = new HashMap<>(); + newAnnotations.put("key", "new-value"); + obj.setMetadata(new V1ObjectMeta().name("target").namespace("ns").annotations(newAnnotations)); + + api.update(obj); + + // After update, obj's metadata = existingObj's metadata + // The existing label value should be present since existing metadata replaces obj's metadata + ArgumentCaptor captor = ArgumentCaptor.forClass(DynamicKubernetesObject.class); + verify(dynApi, times(1)).update(captor.capture()); + Map capturedAnnotations = captor.getValue().getMetadata().getAnnotations(); + assertNotNull(capturedAnnotations); + assertEquals("existing-value", capturedAnnotations.get("key")); + } + + @Test + void updateExistingWithDisjointAnnotationsKeepsExisting() throws SQLException { + // Mirrors updateExistingObjectWithNewLabelsOnlyPreservesAll for annotations: when local and + // existing carry different keys, existing's annotation ends up on the captured update payload. + DynamicKubernetesObject existingObj = new DynamicKubernetesObject(); + existingObj.setApiVersion("v1"); + existingObj.setKind("ConfigMap"); + Map existingAnnotations = new HashMap<>(); + existingAnnotations.put("from-cluster", "e-val"); + existingObj.setMetadata(new V1ObjectMeta().name("target").namespace("ns").annotations(existingAnnotations)); + + DynamicKubernetesApi dynApi = mock(DynamicKubernetesApi.class); + doReturn(dynApi).when(mockContext).dynamic(anyString(), anyString()); + doReturn(successResponse(existingObj)).when(dynApi).get(anyString(), anyString()); + doReturn(successResponse(existingObj)).when(dynApi).update(any(DynamicKubernetesObject.class)); + + K8sYamlApi api = new K8sYamlApi(mockContext); + DynamicKubernetesObject obj = new DynamicKubernetesObject(); + obj.setApiVersion("v1"); + obj.setKind("ConfigMap"); + Map newAnnotations = new HashMap<>(); + newAnnotations.put("from-local", "a-val"); + obj.setMetadata(new V1ObjectMeta().name("target").namespace("ns").annotations(newAnnotations)); + + api.update(obj); + + ArgumentCaptor captor = ArgumentCaptor.forClass(DynamicKubernetesObject.class); + verify(dynApi, times(1)).update(captor.capture()); + Map capturedAnnotations = captor.getValue().getMetadata().getAnnotations(); + assertNotNull(capturedAnnotations); + assertTrue(capturedAnnotations.containsKey("from-cluster"), "Should contain existing annotation"); + } } private K8sYamlApi createRealApi() { diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java index 6cddb0a4..25cf6a40 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java @@ -19,6 +19,7 @@ import com.linkedin.hoptimator.k8s.models.V1alpha1PipelineList; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.when; @@ -85,7 +86,7 @@ void blocksOnExternalWhenSomeAreSelfOwned() throws SQLException { () -> PipelineDependencyChecker.assertNoExternalDependents( api, DB, PATH, "LogicalTable", "self-name")); assertTrue(ex.getMessage().contains("external-pipe")); - assertTrue(!ex.getMessage().contains("owned-pipe"), "self-owned pipeline must not be listed"); + assertFalse(ex.getMessage().contains("owned-pipe"), "self-owned pipeline must not be listed"); } @Test diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java index 68482c5e..eeeaa539 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java @@ -159,26 +159,4 @@ void parseAnnotationHandlesNullAndEmpty() { assertTrue(PipelineDependencyLabels.parseAnnotation("").isEmpty()); assertTrue(PipelineDependencyLabels.parseAnnotation(" , ").isEmpty()); } - - @Test - void stripDependencyLabelsRemovesOnlyPrefixedEntries() { - Map existing = new LinkedHashMap<>(); - existing.put("app", "hoptimator"); // unrelated label, keep - existing.put(PipelineDependencyLabels.labelKey("db", List.of("t")), "db/t"); // strip - existing.put("pipeline", "my-pipeline"); // keep - - Map stripped = PipelineDependencyLabels.stripDependencyLabels(existing); - - assertEquals(2, stripped.size()); - assertTrue(stripped.containsKey("app")); - assertTrue(stripped.containsKey("pipeline")); - assertFalse(stripped.containsKey(PipelineDependencyLabels.labelKey("db", List.of("t")))); - } - - @Test - void stripDependencyLabelsHandlesNull() { - Map result = PipelineDependencyLabels.stripDependencyLabels(null); - assertNotNull(result, "null input must return a non-null empty map, not propagate the null"); - assertTrue(result.isEmpty()); - } } diff --git a/hoptimator-logical/src/test/java/com/linkedin/hoptimator/logical/LogicalTableDeployerTest.java b/hoptimator-logical/src/test/java/com/linkedin/hoptimator/logical/LogicalTableDeployerTest.java index a2d4015a..13ec2547 100644 --- a/hoptimator-logical/src/test/java/com/linkedin/hoptimator/logical/LogicalTableDeployerTest.java +++ b/hoptimator-logical/src/test/java/com/linkedin/hoptimator/logical/LogicalTableDeployerTest.java @@ -117,8 +117,6 @@ private static Properties twoTierProps(String nearlineDb, String offlineDb) { private static K8sContext mockContext() { K8sContext ctx = mock(K8sContext.class); - // Each stub is used by some tests but not all — mark lenient individually so the class - // doesn't need @MockitoSettings(strictness = Strictness.LENIENT). lenient().when(ctx.namespace()).thenReturn("default"); lenient().when(ctx.withOwner(any())).thenReturn(ctx); lenient().when(ctx.withLabel(anyString(), anyString())).thenReturn(ctx); diff --git a/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/ClusterSchemaTest.java b/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/ClusterSchemaTest.java index c3db472d..c5b62272 100644 --- a/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/ClusterSchemaTest.java +++ b/hoptimator-venice/src/test/java/com/linkedin/hoptimator/venice/ClusterSchemaTest.java @@ -194,7 +194,7 @@ void testCreateControllerClientWithNonLocalhostUrl() { @Override protected ControllerClient createControllerClient(String cluster, Optional sslFactory) { // verify the non-localhost branch would be reached (url does not contain localhost) - assertTrue(!properties.getProperty("router.url").contains("localhost")); + assertFalse(properties.getProperty("router.url").contains("localhost")); return mockControllerClient; } }; From 8da3062f30d3bfa45af8faefb8d43233a44bc610 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Fri, 1 May 2026 15:38:47 -0400 Subject: [PATCH 08/15] Add more test coverage --- .../hoptimator/PendingDeleteTest.java | 85 +++++++++++++ .../k8s/PipelineDependencyLabels.java | 1 - .../K8sPipelineDependencyValidatorTest.java | 119 ++++++++++++++++++ .../k8s/K8sValidatorProviderTest.java | 77 ++++++++++++ .../k8s/PipelineDependencyLabelsTest.java | 3 - 5 files changed, 281 insertions(+), 4 deletions(-) create mode 100644 hoptimator-api/src/test/java/com/linkedin/hoptimator/PendingDeleteTest.java create mode 100644 hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidatorTest.java create mode 100644 hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sValidatorProviderTest.java diff --git a/hoptimator-api/src/test/java/com/linkedin/hoptimator/PendingDeleteTest.java b/hoptimator-api/src/test/java/com/linkedin/hoptimator/PendingDeleteTest.java new file mode 100644 index 00000000..4b5b9144 --- /dev/null +++ b/hoptimator-api/src/test/java/com/linkedin/hoptimator/PendingDeleteTest.java @@ -0,0 +1,85 @@ +package com.linkedin.hoptimator; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +class PendingDeleteTest { + + @Test + void singleArgConstructorLeavesSelfOwnerNull() { + Object target = new Object(); + PendingDelete pd = new PendingDelete<>(target); + + assertSame(target, pd.target()); + assertNull(pd.selfOwnerKind()); + assertNull(pd.selfOwnerName()); + } + + @Test + void threeArgConstructorStoresSelfOwner() { + Object target = new Object(); + PendingDelete pd = new PendingDelete<>(target, "LogicalTable", "my-table"); + + assertSame(target, pd.target()); + assertEquals("LogicalTable", pd.selfOwnerKind()); + assertEquals("my-table", pd.selfOwnerName()); + } + + @Test + void threeArgConstructorAcceptsNullSelfOwner() { + Object target = new Object(); + PendingDelete pd = new PendingDelete<>(target, null, null); + + assertNull(pd.selfOwnerKind()); + assertNull(pd.selfOwnerName()); + } + + @Test + void nullTargetThrows() { + assertThrows(NullPointerException.class, () -> new PendingDelete<>(null)); + assertThrows(NullPointerException.class, + () -> new PendingDelete<>(null, "LogicalTable", "my-table")); + } + + @Test + void toStringIncludesTargetAndSelfOwnerWhenPresent() { + PendingDelete pd = new PendingDelete<>("the-target", "LogicalTable", "my-table"); + String s = pd.toString(); + assertTrue(s.contains("the-target"), "toString should include target: " + s); + assertTrue(s.contains("LogicalTable/my-table"), "toString should include kind/name: " + s); + } + + @Test + void toStringOmitsSelfOwnerWhenNull() { + PendingDelete pd = new PendingDelete<>("the-target"); + String s = pd.toString(); + assertTrue(s.contains("the-target")); + assertFalse(s.contains("self="), "toString should not include self= when not set: " + s); + } + + @Test + void toStringOmitsSelfOwnerWhenOnlyOneFieldSet() { + // The toString contract says self= appears only when both kind and name are non-null. + // (The K8sPipelineDependencyChecker.isSelfOwned guard also requires both.) + PendingDelete kindOnly = new PendingDelete<>("t", "LogicalTable", null); + assertFalse(kindOnly.toString().contains("self=")); + + PendingDelete nameOnly = new PendingDelete<>("t", null, "my-table"); + assertFalse(nameOnly.toString().contains("self=")); + } + + @Test + void targetGenericTypeIsPreserved() { + Source source = new Source("db", java.util.List.of("schema", "tbl"), java.util.Map.of()); + PendingDelete pd = new PendingDelete<>(source); + Source unwrapped = pd.target(); + assertEquals("tbl", unwrapped.table()); + } +} diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java index 924e84dc..d9662337 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java @@ -8,7 +8,6 @@ import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; import com.linkedin.hoptimator.Sink; import com.linkedin.hoptimator.Source; diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidatorTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidatorTest.java new file mode 100644 index 00000000..ac94636d --- /dev/null +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidatorTest.java @@ -0,0 +1,119 @@ +package com.linkedin.hoptimator.k8s; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; + +import com.linkedin.hoptimator.Source; +import com.linkedin.hoptimator.Validator; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.nullable; + + +/** + * Unit tests for {@link K8sPipelineDependencyValidator}. The validator is a thin adapter that + * forwards to {@link PipelineDependencyChecker#assertNoExternalDependents} — these tests use + * {@link MockedStatic} on both K8sContext and PipelineDependencyChecker to verify that source + * fields and self-owner fields are forwarded correctly, and that thrown SQLException becomes a + * validation issue rather than propagating. + */ +@ExtendWith(MockitoExtension.class) +class K8sPipelineDependencyValidatorTest { + + @Mock + private MockedStatic contextStatic; + + @Mock + private MockedStatic checkerStatic; + + @Mock + private Connection connection; + + @Mock + private K8sContext context; + + private static Source source() { + return new Source("kafka1", List.of("KAFKA", "my-topic"), Map.of()); + } + + @Test + void validateForwardsSourceFieldsAndSelfOwnerToChecker() { + contextStatic.when(() -> K8sContext.create(connection)).thenReturn(context); + + K8sPipelineDependencyValidator validator = + new K8sPipelineDependencyValidator(source(), "LogicalTable", "my-table"); + Validator.Issues issues = new Validator.Issues("test"); + + validator.validate(issues, connection); + + ArgumentCaptor dbCaptor = ArgumentCaptor.forClass(String.class); + @SuppressWarnings("unchecked") + ArgumentCaptor> pathCaptor = ArgumentCaptor.forClass(List.class); + ArgumentCaptor kindCaptor = ArgumentCaptor.forClass(String.class); + ArgumentCaptor nameCaptor = ArgumentCaptor.forClass(String.class); + + checkerStatic.verify(() -> PipelineDependencyChecker.assertNoExternalDependents( + eq(context), dbCaptor.capture(), pathCaptor.capture(), + kindCaptor.capture(), nameCaptor.capture())); + + assertEquals("kafka1", dbCaptor.getValue()); + assertEquals(List.of("KAFKA", "my-topic"), pathCaptor.getValue()); + assertEquals("LogicalTable", kindCaptor.getValue()); + assertEquals("my-table", nameCaptor.getValue()); + assertTrue(issues.valid()); + } + + @Test + @SuppressWarnings("unchecked") + void validatePassesNullSelfOwnerWhenUnset() { + contextStatic.when(() -> K8sContext.create(connection)).thenReturn(context); + + K8sPipelineDependencyValidator validator = + new K8sPipelineDependencyValidator(source(), null, null); + Validator.Issues issues = new Validator.Issues("test"); + + validator.validate(issues, connection); + + ArgumentCaptor kindCaptor = ArgumentCaptor.forClass(String.class); + ArgumentCaptor nameCaptor = ArgumentCaptor.forClass(String.class); + checkerStatic.verify(() -> PipelineDependencyChecker.assertNoExternalDependents( + eq(context), nullable(String.class), nullable(List.class), + kindCaptor.capture(), nameCaptor.capture())); + + assertNull(kindCaptor.getValue()); + assertNull(nameCaptor.getValue()); + } + + @Test + @SuppressWarnings("unchecked") + void validateRecordsCheckerSqlExceptionAsIssue() { + contextStatic.when(() -> K8sContext.create(connection)).thenReturn(context); + checkerStatic.when(() -> PipelineDependencyChecker.assertNoExternalDependents( + nullable(K8sContext.class), nullable(String.class), nullable(List.class), + nullable(String.class), nullable(String.class))) + .thenThrow(new SQLException("3 active pipeline(s) depend on it: p1, p2, p3")); + + K8sPipelineDependencyValidator validator = + new K8sPipelineDependencyValidator(source(), null, null); + Validator.Issues issues = new Validator.Issues("test"); + + validator.validate(issues, connection); + + assertFalse(issues.valid(), "blocking pipelines should surface as a validation error"); + assertTrue(issues.toString().contains("3 active pipeline"), + "issue message should include the SQLException's text: " + issues); + } +} diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sValidatorProviderTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sValidatorProviderTest.java new file mode 100644 index 00000000..a4cf0189 --- /dev/null +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sValidatorProviderTest.java @@ -0,0 +1,77 @@ +package com.linkedin.hoptimator.k8s; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; + +import org.junit.jupiter.api.Test; + +import com.linkedin.hoptimator.PendingDelete; +import com.linkedin.hoptimator.Source; +import com.linkedin.hoptimator.Validator; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +class K8sValidatorProviderTest { + + private static Source testSource() { + return new Source("kafka1", java.util.List.of("KAFKA", "my-topic"), Map.of()); + } + + @Test + void returnsDependencyValidatorForPendingDeleteOfSource() { + K8sValidatorProvider provider = new K8sValidatorProvider(); + PendingDelete pd = new PendingDelete<>(testSource()); + + Collection validators = provider.validators(pd, null); + + assertEquals(1, validators.size()); + assertInstanceOf(K8sPipelineDependencyValidator.class, validators.iterator().next()); + } + + @Test + void returnsDependencyValidatorWhenSelfOwnerIsSet() { + // Self-owner fields are stored on the validator; this exercises the (kind, name) plumbing + // through the provider boundary. + K8sValidatorProvider provider = new K8sValidatorProvider(); + PendingDelete pd = new PendingDelete<>(testSource(), "LogicalTable", "my-table"); + + Collection validators = provider.validators(pd, null); + + assertEquals(1, validators.size()); + assertInstanceOf(K8sPipelineDependencyValidator.class, validators.iterator().next()); + } + + @Test + void returnsEmptyForRawSourceWithoutPendingDeleteWrapper() { + // The validator only fires for a delete-intent signal — not for a plain Source. Other callers + // of ValidationService.validate(source, ...) must NOT trigger the K8s pipeline lookup. + K8sValidatorProvider provider = new K8sValidatorProvider(); + + Collection validators = provider.validators(testSource(), null); + + assertTrue(validators.isEmpty()); + } + + @Test + void returnsEmptyForPendingDeleteOfNonSourceTarget() { + K8sValidatorProvider provider = new K8sValidatorProvider(); + PendingDelete pd = new PendingDelete<>("not-a-source"); + + Collection validators = provider.validators(pd, null); + + assertTrue(validators.isEmpty()); + } + + @Test + void returnsEmptyForUnrelatedTypes() { + K8sValidatorProvider provider = new K8sValidatorProvider(); + + assertTrue(provider.validators("just-a-string", null).isEmpty()); + assertTrue(provider.validators(42, null).isEmpty()); + assertTrue(provider.validators(Collections.emptyList(), null).isEmpty()); + } +} diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java index eeeaa539..b5cd6afb 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java @@ -2,8 +2,6 @@ import java.util.Arrays; import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; import java.util.Set; @@ -15,7 +13,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; From b7fc7498b03fda855133c650118bd90e2ab7c4e3 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Fri, 8 May 2026 12:58:29 -0400 Subject: [PATCH 09/15] Split depends-on annotation into directional sources/sink MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Pipelines previously stamped a single `depends-on` annotation listing every source and sink undifferentiated. The dep-guard collision check worked on this, but it loses the source/sink direction information needed for visualization. Replace the unified annotation with two directional annotations: hoptimator.linkedin.com/depends-on-sources: ,,... hoptimator.linkedin.com/depends-on-sink: The dep-guard's annotationConfirms now reads sources or sink as the collision-guard set — same correctness guarantee, no semantic change for the dep-guard. The split unlocks directional rendering for the upcoming pipeline visualizer. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../hoptimator/k8s/K8sPipelineDeployer.java | 10 ++++- .../k8s/PipelineDependencyChecker.java | 22 ++++++---- .../k8s/PipelineDependencyLabels.java | 41 +++++++++++++------ .../k8s/K8sPipelineDeployerTest.java | 13 +++--- .../k8s/PipelineDependencyCheckerTest.java | 4 +- .../k8s/PipelineDependencyLabelsTest.java | 34 ++++++++------- 6 files changed, 79 insertions(+), 45 deletions(-) diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java index 0a9cfedf..551dfee3 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java @@ -52,8 +52,14 @@ protected V1alpha1Pipeline toK8sObject() throws SQLException { if (!labels.isEmpty()) { meta.setLabels(new HashMap<>(labels)); Map annotations = new HashMap<>(); - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY, - PipelineDependencyLabels.annotationFor(sources, sink)); + String sourcesAnno = PipelineDependencyLabels.sourcesAnnotation(sources); + if (!sourcesAnno.isEmpty()) { + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, sourcesAnno); + } + String sinkAnno = PipelineDependencyLabels.sinkAnnotation(sink); + if (sinkAnno != null) { + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SINK, sinkAnno); + } meta.setAnnotations(annotations); } return new V1alpha1Pipeline().kind(K8sApiEndpoints.PIPELINES.kind()) diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java index ee9a6b64..8cbcb2d6 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java @@ -4,7 +4,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.Set; import io.kubernetes.client.openapi.models.V1ObjectMeta; import io.kubernetes.client.openapi.models.V1OwnerReference; @@ -20,9 +19,11 @@ * is about to delete. * *

    The lookup is a label-selector list against the Pipeline CRD group, so it is O(matches) on - * the wire — not a full scan. Each candidate is then cross-checked against the - * {@link PipelineDependencyLabels#ANNOTATION_KEY} annotation to rule out the (rare) case of a - * hash collision in the label slug. + * the wire — not a full scan. Each candidate is then cross-checked against the union of the + * {@link PipelineDependencyLabels#ANNOTATION_KEY_SOURCES sources} and + * {@link PipelineDependencyLabels#ANNOTATION_KEY_SINK sink} annotations to rule out hash + * collisions in the label slug and stale labels left over from a prior version of the pipeline's + * SQL ({@link K8sApi#update}'s additive label merge can leak old {@code depends-on-*} keys). * *

    Pipelines owned (directly) by {@code (selfOwnerKind, selfOwnerName)} are excluded from the * blocker list: those pipelines will be cascade-deleted alongside the parent resource, so counting @@ -91,12 +92,15 @@ private static boolean annotationConfirms(V1alpha1Pipeline pipeline, String iden if (meta == null || meta.getAnnotations() == null) { return true; // pre-labeling pipeline — conservatively trust the label match } - String annotation = meta.getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY); - if (annotation == null) { - return true; // same — no annotation to cross-check against + String sourcesAnno = meta.getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES); + String sinkAnno = meta.getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SINK); + if (sourcesAnno == null && sinkAnno == null) { + return true; // same — no annotations to cross-check against } - Set listed = PipelineDependencyLabels.parseAnnotation(annotation); - return listed.contains(identifier); + if (sourcesAnno != null && PipelineDependencyLabels.parseAnnotation(sourcesAnno).contains(identifier)) { + return true; + } + return identifier.equals(sinkAnno); } /** diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java index d9662337..07db426e 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java @@ -14,7 +14,7 @@ /** - * Computes the labels and annotation that encode a Pipeline CRD's dependency edges. + * Computes the labels and annotations that encode a Pipeline CRD's dependency edges. * *

    Every source and sink a pipeline references is recorded as a label: * {@code hoptimator.linkedin.com/depends-on-: "_"} where @@ -22,13 +22,27 @@ * The hash keeps label keys within Kubernetes's 63-character name limit for arbitrary paths, * and lets {@code K8sApi.select} filter pipelines by dependency on the server. * - *

    A collision-guard annotation ({@code ANNOTATION_KEY}) lists all logical identifiers verbatim, - * so the delete-time check can distinguish a real dependency match from a rare hash collision. + *

    Two annotations preserve the directional information the labels lose: + *

      + *
    • {@link #ANNOTATION_KEY_SOURCES} — comma-separated list of source identifiers verbatim.
    • + *
    • {@link #ANNOTATION_KEY_SINK} — the single sink identifier verbatim.
    • + *
    + * Together they serve two purposes: + *
      + *
    1. Collision guard for the delete-time dependency checker — a label match is only + * trusted when the resource appears in either annotation, so slug collisions and stale + * labels (from {@link K8sApi#update}'s additive label merge) can't produce false positives.
    2. + *
    3. Direction recovery for visualization — the renderer draws source → pipeline → sink + * arrows from the split.
    4. + *
    */ public final class PipelineDependencyLabels { static final String LABEL_PREFIX = "hoptimator.linkedin.com/depends-on-"; - public static final String ANNOTATION_KEY = "hoptimator.linkedin.com/depends-on"; + /** Annotation listing only source identifiers. */ + public static final String ANNOTATION_KEY_SOURCES = "hoptimator.linkedin.com/depends-on-sources"; + /** Annotation holding the single sink identifier. */ + public static final String ANNOTATION_KEY_SINK = "hoptimator.linkedin.com/depends-on-sink"; private static final int SLUG_LENGTH = 16; // 64 bits of SHA-256 → ~1 in 1.8e19 collisions private static final int MAX_LABEL_VALUE = 63; @@ -78,22 +92,23 @@ public static Map labelsFor(Collection sources, Sink sin return labels; } - /** - * Collision-guard annotation value — comma-separated list of full source and sink identifiers, - * deduplicated and not truncated. The delete-time check cross-references this annotation after - * the label selector narrows the candidate set. - */ - public static String annotationFor(Collection sources, Sink sink) { + /** Comma-separated list of source identifiers (no sink). Empty string if no sources. */ + public static String sourcesAnnotation(Collection sources) { Set ids = new LinkedHashSet<>(); for (Source src : sources) { ids.add(identifier(src.database(), src.path())); } - if (sink != null) { - ids.add(identifier(sink.database(), sink.path())); - } return String.join(",", ids); } + /** Single sink identifier, or {@code null} if there is no sink. */ + public static String sinkAnnotation(Sink sink) { + if (sink == null) { + return null; + } + return identifier(sink.database(), sink.path()); + } + /** Parses the collision-guard annotation back into the set of identifiers it encoded. */ public static Set parseAnnotation(String annotation) { Set out = new LinkedHashSet<>(); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java index a18122ed..08a9f1a2 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java @@ -70,7 +70,7 @@ void stampsDependencyLabelsForSourcesAndSink() throws SQLException { } @Test - void stampsCollisionGuardAnnotation() throws SQLException { + void stampsDirectionalAnnotations() throws SQLException { K8sPipelineDeployer deployer = new K8sPipelineDeployer( "p1", List.of("spec"), "SELECT 1", Collections.singletonList(src("kafka", "topic")), @@ -79,9 +79,12 @@ void stampsCollisionGuardAnnotation() throws SQLException { V1alpha1Pipeline pipeline = deployer.toK8sObject(); Map annotations = pipeline.getMetadata().getAnnotations(); - String annotation = annotations.get(PipelineDependencyLabels.ANNOTATION_KEY); - assertNotNull(annotation); - assertTrue(annotation.contains("kafka_topic")); - assertTrue(annotation.contains("mysql_outbox")); + String sources = annotations.get(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES); + String sink = annotations.get(PipelineDependencyLabels.ANNOTATION_KEY_SINK); + assertNotNull(sources); + assertNotNull(sink); + assertTrue(sources.contains("kafka_topic"), + "sources annotation should list the kafka source: " + sources); + assertEquals("mysql_outbox", sink, "sink annotation holds the single identifier verbatim"); } } diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java index 25cf6a40..6147686d 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java @@ -42,8 +42,10 @@ private static V1alpha1Pipeline pipeline(String name, String ownerKind, String o meta.addOwnerReferencesItem(new V1OwnerReference().kind(ownerKind).name(ownerName)); } if (annotationValue != null) { + // Treat the test's single value as the sources annotation. The cross-check now reads + // sources ∪ sink, so writing to either one is sufficient to confirm the identifier. Map annotations = new HashMap<>(); - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY, annotationValue); + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, annotationValue); meta.setAnnotations(annotations); } return new V1alpha1Pipeline().metadata(meta); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java index b5cd6afb..ce126558 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java @@ -13,6 +13,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -121,33 +122,36 @@ void labelValueIsKubernetesLabelValueCompliant() { } @Test - void annotationForListsAllIdentifiers() { - // Sources and the sink — both edges are recorded so the delete-time check can disambiguate - // a real dependency from a hash collision regardless of which side matched. - String annotation = PipelineDependencyLabels.annotationFor( - Arrays.asList(src("kafka", "a"), src("venice", "b")), - sink("mysql", "c")); + void sourcesAnnotationListsOnlySources() { + String annotation = PipelineDependencyLabels.sourcesAnnotation( + Arrays.asList(src("kafka", "a"), src("venice", "b"))); assertTrue(annotation.contains("kafka_a")); assertTrue(annotation.contains("venice_b")); - assertTrue(annotation.contains("mysql_c")); + assertFalse(annotation.contains("mysql_c"), "sinks must not appear in sources annotation"); } @Test - void annotationForDeduplicatesAndOmitsNullSink() { - String annotation = PipelineDependencyLabels.annotationFor( - Arrays.asList(src("db", "t"), src("db", "t")), null); + void sourcesAnnotationDeduplicatesIdenticalSources() { + String annotation = PipelineDependencyLabels.sourcesAnnotation( + Arrays.asList(src("db", "t"), src("db", "t"))); assertEquals("db_t", annotation); } @Test - void parseAnnotationRoundtrip() { - String annotation = PipelineDependencyLabels.annotationFor( - Arrays.asList(src("a", "1"), src("b", "2")), sink("c", "3")); + void sinkAnnotationReturnsSingleIdentifierOrNull() { + assertEquals("mysql_c", PipelineDependencyLabels.sinkAnnotation(sink("mysql", "c"))); + assertNull(PipelineDependencyLabels.sinkAnnotation(null), + "sink annotation should be null when there's no sink"); + } + + @Test + void parseAnnotationRoundtripFromSourcesValue() { + String annotation = PipelineDependencyLabels.sourcesAnnotation( + Arrays.asList(src("a", "1"), src("b", "2"))); Set parsed = PipelineDependencyLabels.parseAnnotation(annotation); - assertEquals(3, parsed.size()); + assertEquals(2, parsed.size()); assertTrue(parsed.contains("a_1")); assertTrue(parsed.contains("b_2")); - assertTrue(parsed.contains("c_3")); } @Test From 0d211380197a51f88c380742b3cff539e467f027 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Tue, 12 May 2026 09:34:01 -0400 Subject: [PATCH 10/15] Add typed Source/Sink to Trigger and stamp depends-on labels MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Two related changes that bring triggers into the depends-on label index so the pre-delete dep-guard can find them. Trigger API: - Replace the old `(String database, List path)` pair with a typed `Source source` field (existing hoptimator-api type). Drop the convenience getters database() / path() / table() / schema() — callers go through trigger.source().X() for symmetry with the new Sink. - Add an optional `Sink sink` field for bridging-tier triggers (LogicalTableDeployer's offline → online reverse-ETL flow), so the deployer can stamp a depends-on-sink annotation in addition to the source side. - Source is nullable for DROP / PAUSE / RESUME paths that only need the trigger name. K8sTriggerDeployer stamping: - On both the toK8sObject and the partial-update paths, stamp: depends-on-: annotation depends-on-sources: and when sink is set: depends-on-: annotation depends-on-sink: LogicalTableDeployer wiring: - Pass `offlineSource` directly as the trigger's Source and a Sink derived from `onlineSource` (when present) as the trigger's Sink. HoptimatorDdlExecutor: - Resolve the target table's database name the same way DROP TABLE resolves it (HoptimatorJdbcTable / TemporaryTable unwrap), so user-created `CREATE TRIGGER ... ON .` triggers participate in the dep-guard. Without this, Trigger.source was null and K8sTriggerDeployer skipped label stamping — a trigger could outlive its source silently. Tests: - K8sTriggerDeployerTest gains updateStampsSinkLabelWhenTriggerCarriesASink pinning that a Trigger carrying a Sink stamps both source-side and sink-side depends-on labels on the partial-update path. - TriggerTest covers the new accessor shape (source(), sink(), null source for the bare-name case). - LogicalTableDeployerTest asserts trigger.source() / trigger.sink() instead of the removed convenience getters. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../java/com/linkedin/hoptimator/Trigger.java | 52 ++++---- .../com/linkedin/hoptimator/TriggerTest.java | 29 ++-- .../jdbc/HoptimatorDdlExecutor.java | 30 ++++- .../hoptimator/k8s/K8sTriggerDeployer.java | 76 ++++++++++- .../k8s/K8sTriggerDeployerTest.java | 126 ++++++++++++------ .../logical/LogicalTableDeployer.java | 9 +- .../logical/LogicalTableDeployerTest.java | 16 ++- 7 files changed, 251 insertions(+), 87 deletions(-) diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Trigger.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Trigger.java index 1eabf09a..1aefdbb7 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Trigger.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Trigger.java @@ -1,35 +1,44 @@ package com.linkedin.hoptimator; -import java.util.List; import java.util.Map; public class Trigger implements Deployable { public static final String PAUSED_OPTION = "paused"; + private final String name; private final UserJob job; - private final List path; private final String cronSchedule; private final Map options; + private final Source source; + private final Sink sink; + + public Trigger(String name, UserJob job, String cronSchedule, Map options, + Source source) { + this(name, job, cronSchedule, options, source, null); + } - public Trigger(String name, UserJob job, List path, String cronSchedule, - Map options) { + /** + * Variant accepting an optional downstream sink. Set by {@code LogicalTableDeployer} on + * bridging-tier triggers so the visualizer can render the directional flow + * {@code source -.-> trigger -.-> sink}, and so the dep-guard finds the sink-side + * dependency. {@code null} for user-created triggers (no declared sink). + */ + public Trigger(String name, UserJob job, String cronSchedule, Map options, + Source source, Sink sink) { this.name = name; this.job = job; - this.path = path; this.cronSchedule = cronSchedule; this.options = options; + this.source = source; + this.sink = sink; } public String name() { return name; } - public List path() { - return path; - } - public UserJob job() { return job; } @@ -38,27 +47,24 @@ public String cronSchedule() { return cronSchedule; } - public String table() { - return path.get(path.size() - 1); - } - - /** - * Returns the schema name if present. - */ - public String schema() { - return path.size() >= 2 ? path.get(path.size() - 2) : null; - } - public Map options() { return options; } - private String pathString() { - return String.join(".", path); + /** Upstream source the trigger fires on, or {@code null} when only the name is known + * (e.g. during DROP TRIGGER / PAUSE / RESUME, which only need to look up the existing CRD). */ + public Source source() { + return source; + } + + /** Downstream sink the trigger's job writes to, or {@code null} when the trigger has no declared sink. */ + public Sink sink() { + return sink; } @Override public String toString() { - return "Trigger[" + name() + ", " + pathString() + "]"; + String path = source == null ? "" : String.join(".", source.path()); + return "Trigger[" + name + ", " + path + "]"; } } diff --git a/hoptimator-api/src/test/java/com/linkedin/hoptimator/TriggerTest.java b/hoptimator-api/src/test/java/com/linkedin/hoptimator/TriggerTest.java index 29558823..2dbec09e 100644 --- a/hoptimator-api/src/test/java/com/linkedin/hoptimator/TriggerTest.java +++ b/hoptimator-api/src/test/java/com/linkedin/hoptimator/TriggerTest.java @@ -2,10 +2,12 @@ import org.junit.jupiter.api.Test; +import java.util.Collections; import java.util.List; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -15,26 +17,37 @@ class TriggerTest { void testAccessors() { UserJob userJob = new UserJob("ns", "jobName"); Map options = Map.of("paused", "true"); - Trigger trigger = new Trigger("myTrigger", userJob, List.of("schema", "table"), "0 * * * *", options); + Source source = new Source("db", List.of("schema", "table"), Collections.emptyMap()); + Trigger trigger = new Trigger("myTrigger", userJob, "0 * * * *", options, source); assertEquals("myTrigger", trigger.name()); assertEquals(userJob, trigger.job()); - assertEquals(List.of("schema", "table"), trigger.path()); assertEquals("0 * * * *", trigger.cronSchedule()); assertEquals(options, trigger.options()); - assertEquals("table", trigger.table()); - assertEquals("schema", trigger.schema()); + assertNotNull(trigger.source()); + assertEquals(List.of("schema", "table"), trigger.source().path()); + assertEquals("table", trigger.source().table()); + assertEquals("schema", trigger.source().schema()); + assertNull(trigger.sink()); } @Test - void testSchemaReturnsNullForSingleElementPath() { - Trigger trigger = new Trigger("t", null, List.of("table"), null, Map.of()); - assertNull(trigger.schema()); + void testNullSourceAccessor() { + Trigger trigger = new Trigger("t", null, null, Map.of(), null); + assertNull(trigger.source()); + assertNull(trigger.sink()); } @Test void testToString() { - Trigger trigger = new Trigger("myTrig", null, List.of("a", "b"), null, Map.of()); + Source source = new Source("db", List.of("a", "b"), Collections.emptyMap()); + Trigger trigger = new Trigger("myTrig", null, null, Map.of(), source); assertEquals("Trigger[myTrig, a.b]", trigger.toString()); } + + @Test + void testToStringWithoutSource() { + Trigger trigger = new Trigger("myTrig", null, null, Map.of(), null); + assertEquals("Trigger[myTrig, ]", trigger.toString()); + } } diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index 18e229e7..d87f8ef5 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -229,7 +229,11 @@ public void execute(SqlCreateTrigger create, CalcitePrepare.Context context) { String cronSchedule = create.cron != null ? ((SqlLiteral) create.cron).getValueAs(String.class) : null; UserJob job = new UserJob(jobNamespace, jobName); - Trigger trigger = new Trigger(name, job, targetPath, cronSchedule, options); + // Resolve the underlying database name the target table belongs to so the deployer can + // stamp depends-on labels: same identifier shape as Pipelines, so the dep-guard finds the + // trigger when somebody tries to drop the source it points at. + Source source = new Source(databaseOf(target), targetPath, Collections.emptyMap()); + Trigger trigger = new Trigger(name, job, cronSchedule, options, source); Collection deployers = null; try { @@ -255,6 +259,26 @@ public void execute(SqlCreateTrigger create, CalcitePrepare.Context context) { } } + /** + * Best-effort lookup of the database name that owns a Calcite Table. Mirrors what the + * {@code DROP TABLE} path does (lines 432–437) so triggers and tables agree on the same + * {@code (database, path)} identifier, which is what the dep-guard's depends-on labels are + * keyed on. Returns {@code null} for table types that don't carry a database (e.g. ad-hoc + * Calcite views) — depends-on labels are skipped in that case, so the trigger simply won't + * participate in the dep-guard. + */ + private static String databaseOf(Table target) { + if (target instanceof HoptimatorJdbcTable) { + HoptimatorJdbcSchema jdbcSchema = + (HoptimatorJdbcSchema) ((HoptimatorJdbcTable) target).jdbcTable().jdbcSchema; + return jdbcSchema.databaseName(); + } + if (target instanceof TemporaryTable) { + return ((TemporaryTable) target).databaseName(); + } + return null; + } + // N.B. originally copy-pasted from Apache Calcite /** Executes a {@code CREATE TABLE} command. */ @@ -307,7 +331,7 @@ public void execute(SqlDropTrigger drop, CalcitePrepare.Context context) { } String name = drop.name.names.get(0); - Trigger trigger = new Trigger(name, null, new ArrayList<>(), null, new HashMap<>()); + Trigger trigger = new Trigger(name, null, null, new HashMap<>(), null); Collection deployers = null; try { @@ -344,7 +368,7 @@ private void updateTriggerPausedState(SqlNode sqlNode, SqlIdentifier triggerName Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, String.valueOf(paused)); - Trigger trigger = new Trigger(name, null, new ArrayList<>(), null, options); + Trigger trigger = new Trigger(name, null, null, options, null); Collection deployers = null; try { diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java index 29289593..aabdd276 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java @@ -1,5 +1,7 @@ package com.linkedin.hoptimator.k8s; +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.Trigger; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplate; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplateList; @@ -17,6 +19,9 @@ public class K8sTriggerDeployer extends K8sDeployer { + /** Annotation carrying the JobTemplate name (e.g. {@code retl-job-template}) used to render this trigger. */ + static final String JOB_TEMPLATE_ANNOTATION = "hoptimator.linkedin.com/job-template"; + private final K8sContext context; private final Trigger trigger; private final K8sApi triggerApi; @@ -60,12 +65,46 @@ public void update() throws SQLException { existingTrigger.spec(spec); } spec.setPaused(targetPaused); + // Refresh dependency-tracking labels and annotation here too — without this, the partial + // update path (used when the LogicalTable is re-applied) would leave triggers with stale + // or missing depends-on metadata, breaking the visualizer's reverse lookup. + stampDependencyMetadata(existingTrigger); triggerApi.update(existingTrigger); return; } super.update(); } + private void stampDependencyMetadata(V1alpha1TableTrigger target) { + V1ObjectMeta meta = target.getMetadata(); + if (meta == null) { + meta = new V1ObjectMeta(); + target.metadata(meta); + } + Map labels = meta.getLabels() != null ? meta.getLabels() : new HashMap<>(); + Map annotations = meta.getAnnotations() != null ? meta.getAnnotations() : new HashMap<>(); + Source source = trigger.source(); + if (source != null && source.database() != null) { + String labelKey = PipelineDependencyLabels.labelKey(source.database(), source.path()); + String identifier = PipelineDependencyLabels.identifier(source.database(), source.path()); + labels.put(labelKey, identifier.length() <= 63 ? identifier : identifier.substring(0, 63)); + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, identifier); + } + Sink sink = trigger.sink(); + if (sink != null) { + String sinkLabelKey = PipelineDependencyLabels.labelKey(sink.database(), sink.path()); + String sinkIdentifier = PipelineDependencyLabels.identifier(sink.database(), sink.path()); + labels.put(sinkLabelKey, + sinkIdentifier.length() <= 63 ? sinkIdentifier : sinkIdentifier.substring(0, 63)); + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SINK, sinkIdentifier); + } + if (trigger.job() != null) { + annotations.put(JOB_TEMPLATE_ANNOTATION, trigger.job().name()); + } + meta.setLabels(labels); + meta.setAnnotations(annotations); + } + @Override public void delete() throws SQLException { String canonicalName = K8sUtils.canonicalizeName(trigger.name()); @@ -78,9 +117,10 @@ public void delete() throws SQLException { @Override protected V1alpha1TableTrigger toK8sObject() throws SQLException { + Source source = trigger.source(); String name = K8sUtils.canonicalizeName(trigger.name(), trigger.job().name()); String triggerName = K8sUtils.canonicalizeName(trigger.name()); - String viewName = K8sUtils.canonicalizeName(trigger.path()); + String viewName = source != null ? K8sUtils.canonicalizeName(source.path()) : triggerName; String jobName = K8sUtils.canonicalizeName(trigger.job().name()); String jobNamespace = trigger.job().namespace() != null ? trigger.job().namespace() : context.namespace(); @@ -91,12 +131,32 @@ protected V1alpha1TableTrigger toK8sObject() throws SQLException { .with("trigger", triggerName) .with("job", jobName) .with("schedule", trigger.cronSchedule()) - .with("table", trigger.table()) - .with("schema", trigger.schema()) + .with("table", source != null ? source.table() : null) + .with("schema", source != null ? source.schema() : null) .with(properties); V1alpha1JobTemplate jobTemplate = jobTemplateApi.get(jobNamespace, jobName); Map labels = new HashMap<>(); labels.put("view", viewName); // a corresponding view object may or may not exist. + Map annotations = new HashMap<>(); + // Stamp depends-on labels so the visualizer (and the dep-guard) can find triggers via + // label selector. The Trigger's source is the upstream table the job reads from. When the + // trigger carries a Sink (set by LogicalTableDeployer for bridging triggers), we + // additionally stamp the sink — that's what makes reverse-ETL flows render as connectors. + if (source != null && source.database() != null) { + String labelKey = PipelineDependencyLabels.labelKey(source.database(), source.path()); + String identifier = PipelineDependencyLabels.identifier(source.database(), source.path()); + labels.put(labelKey, identifier.length() <= 63 ? identifier : identifier.substring(0, 63)); + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, identifier); + } + Sink sink = trigger.sink(); + if (sink != null) { + String sinkLabelKey = PipelineDependencyLabels.labelKey(sink.database(), sink.path()); + String sinkIdentifier = PipelineDependencyLabels.identifier(sink.database(), sink.path()); + labels.put(sinkLabelKey, + sinkIdentifier.length() <= 63 ? sinkIdentifier : sinkIdentifier.substring(0, 63)); + annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SINK, sinkIdentifier); + } + annotations.put(JOB_TEMPLATE_ANNOTATION, trigger.job().name()); String template = jobTemplate.getSpec().getYaml(); String rendered = new Template.SimpleTemplate(template).render(env); Map jobProps = new HashMap<>(); @@ -106,8 +166,8 @@ protected V1alpha1TableTrigger toK8sObject() throws SQLException { } }); V1alpha1TableTriggerSpec spec = new V1alpha1TableTriggerSpec() - .schema(trigger.schema()) - .table(trigger.table()) + .schema(source != null ? source.schema() : null) + .table(source != null ? source.table() : null) .schedule(trigger.cronSchedule()) .yaml(rendered); if (!jobProps.isEmpty()) { @@ -116,10 +176,14 @@ protected V1alpha1TableTrigger toK8sObject() throws SQLException { if (trigger.options().containsKey(Trigger.PAUSED_OPTION)) { spec.paused("true".equals(trigger.options().get(Trigger.PAUSED_OPTION))); } + V1ObjectMeta meta = new V1ObjectMeta().name(triggerName).labels(labels); + if (annotations != null) { + meta.setAnnotations(annotations); + } return new V1alpha1TableTrigger() .kind(K8sApiEndpoints.TABLE_TRIGGERS.kind()) .apiVersion(K8sApiEndpoints.TABLE_TRIGGERS.apiVersion()) - .metadata(new V1ObjectMeta().name(triggerName).labels(labels)) + .metadata(meta) .spec(spec); } } diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java index 56e0c10d..31003784 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java @@ -1,5 +1,7 @@ package com.linkedin.hoptimator.k8s; +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.Trigger; import com.linkedin.hoptimator.UserJob; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplate; @@ -98,8 +100,7 @@ void updateWithPausedOptionPausesTrigger() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -117,8 +118,7 @@ void updateWithPausedOptionUnpausesTrigger() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "false"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -136,8 +136,7 @@ void updateWithPausedOptionCreatesSpecIfNull() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -147,12 +146,76 @@ void updateWithPausedOptionCreatesSpecIfNull() throws SQLException { assertTrue(existing.getSpec().getPaused()); } + @Test + void updateWithPausedOptionAlsoStampsDependsOnLabelsWhenDatabaseSet() throws SQLException { + // The partial-update path (paused-only) used to skip toK8sObject and never refresh the + // depends-on metadata. Pin down that re-applying the LogicalTable through this path now + // stamps the labels/annotation so visualization (and the dep-guard reverse lookup) works. + V1alpha1TableTrigger existing = new V1alpha1TableTrigger() + .metadata(new V1ObjectMeta().name("mytrigger")) + .spec(new V1alpha1TableTriggerSpec().paused(true)); + triggers.add(existing); + + Map options = new HashMap<>(); + options.put(Trigger.PAUSED_OPTION, "true"); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), null, options, new Source("mysql-db", Arrays.asList("MYSQL", "testdb", "events"), Collections.emptyMap())); + + K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); + + deployer.update(); + + String expectedLabel = PipelineDependencyLabels.labelKey( + "mysql-db", Arrays.asList("MYSQL", "testdb", "events")); + String expectedIdentifier = PipelineDependencyLabels.identifier( + "mysql-db", Arrays.asList("MYSQL", "testdb", "events")); + assertNotNull(existing.getMetadata().getLabels(), "labels must be set after partial update"); + assertTrue(existing.getMetadata().getLabels().containsKey(expectedLabel), + "depends-on label must be stamped on the partial-update path: " + existing.getMetadata().getLabels()); + assertNotNull(existing.getMetadata().getAnnotations(), "annotations must be set"); + assertEquals(expectedIdentifier, + existing.getMetadata().getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES)); + } + + @Test + void updateStampsSinkLabelWhenTriggerCarriesASink() throws SQLException { + // Bridging-tier triggers (LogicalTableDeployer's offline→online flow) carry a Sink in + // addition to their source. Pin that the partial-update path stamps both source and sink + // depends-on labels so the dep-guard reverse lookup finds the trigger from either end. + V1alpha1TableTrigger existing = new V1alpha1TableTrigger() + .metadata(new V1ObjectMeta().name("mytrigger")) + .spec(new V1alpha1TableTriggerSpec().paused(true)); + triggers.add(existing); + + Map options = new HashMap<>(); + options.put(Trigger.PAUSED_OPTION, "true"); + Source source = new Source("hdfs-db", Arrays.asList("HDFS", "events"), Collections.emptyMap()); + Sink sink = new Sink("venice-db", Arrays.asList("VENICE", "events"), Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), null, options, source, sink); + + K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); + deployer.update(); + + String sourceLabel = PipelineDependencyLabels.labelKey( + "hdfs-db", Arrays.asList("HDFS", "events")); + String sinkLabel = PipelineDependencyLabels.labelKey( + "venice-db", Arrays.asList("VENICE", "events")); + String sinkIdentifier = PipelineDependencyLabels.identifier( + "venice-db", Arrays.asList("VENICE", "events")); + + assertTrue(existing.getMetadata().getLabels().containsKey(sourceLabel), + "source-side depends-on label must be stamped: " + existing.getMetadata().getLabels()); + assertTrue(existing.getMetadata().getLabels().containsKey(sinkLabel), + "sink-side depends-on label must be stamped: " + existing.getMetadata().getLabels()); + assertEquals(sinkIdentifier, + existing.getMetadata().getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SINK), + "depends-on-sink annotation must record the sink identifier verbatim"); + } + @Test void updateWithPausedOptionThrowsWhenTriggerNotFound() { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -166,8 +229,7 @@ void deleteRemovesTrigger() throws SQLException { .spec(new V1alpha1TableTriggerSpec()); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -178,8 +240,7 @@ void deleteRemovesTrigger() throws SQLException { @Test void deleteThrowsWhenTriggerNotFound() { - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -194,8 +255,7 @@ void toK8sObjectBuildsCorrectTrigger() throws SQLException { + "metadata:\n name: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "TABLE"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -213,8 +273,7 @@ void toK8sObjectWithNoJobPropertiesHasNullJobProperties() throws SQLException { jobTemplates.add(jobTemplate); // No job.properties.* options — spec should NOT have jobProperties set - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "TABLE"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -234,8 +293,7 @@ void toK8sObjectWithJobPropertiesIncludesThemInSpec() throws SQLException { Map options = new HashMap<>(); options.put("job.properties.parallelism", "4"); options.put("job.properties.restart-strategy", "never"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "TABLE"), "0 * * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -258,8 +316,7 @@ void toK8sObjectOptionsPutAllIncludedInEnvironment() throws SQLException { Map options = new HashMap<>(); options.put("someKey", "someValue"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "MY_TABLE"), "5 4 * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "5 4 * * *", options, new Source(null, Arrays.asList("SCHEMA", "MY_TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -282,8 +339,7 @@ void toK8sObjectForEachAppliesJobPropertiesFilter() throws SQLException { options.put("job.properties.key1", "val1"); options.put("job.properties.key2", "val2"); options.put("other.option", "ignored"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "TABLE"), "0 * * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -305,8 +361,7 @@ void updateWithPausedOptionCallsApiUpdate() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -326,8 +381,7 @@ void updateWithChangedSpecCallsSuperUpdate() throws SQLException { jobTemplates.add(jobTemplate); // No PAUSED_OPTION → should fall through to super.update() - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "TABLE"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); // super.update() calls api.update() via K8sDeployer; FakeK8sApi.update adds to list @@ -339,8 +393,7 @@ void updateWithChangedSpecCallsSuperUpdate() throws SQLException { @Test void deleteOnNonExistingTriggerThrowsSqlException() { // Graceful handling when trigger not found - Trigger trigger = new Trigger("NONEXISTENT", new UserJob(null, "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("NONEXISTENT", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -360,8 +413,7 @@ void updatePreservesPausedWhenOptionsHaveNoPausedOption() throws SQLException { .spec(new V1alpha1TableTriggerSpec().paused(true)); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -378,8 +430,7 @@ void updatePreservesUnpausedWhenOptionsHaveNoPausedOption() throws SQLException .spec(new V1alpha1TableTriggerSpec().paused(false)); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), - Arrays.asList("schema", "table"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -399,7 +450,7 @@ void updateWithPausedOptionFalseUnpausesAlreadyPausedTrigger() throws SQLExcepti Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "false"); - Trigger trigger = new Trigger("MY_TRIGGER", null, new ArrayList<>(), null, options); + Trigger trigger = new Trigger("MY_TRIGGER", null, null, options, new Source(null, new ArrayList<>(), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -418,8 +469,7 @@ void updateFallsThroughToSuperUpdateWhenNoExistingAndNoPausedOption() throws SQL .spec(new V1alpha1JobTemplateSpec().yaml("template: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "TABLE"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -443,8 +493,7 @@ void updateFallsThroughToSuperUpdateWhenExistingHasNullPaused() throws SQLExcept .spec(new V1alpha1JobTemplateSpec().yaml("template: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "TABLE"), "0 * * * *", Collections.emptyMap()); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -466,8 +515,7 @@ void toK8sObjectSetsSpecPausedWhenPausedOptionTrue() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), - Arrays.asList("SCHEMA", "TABLE"), "0 * * * *", options); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); diff --git a/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java b/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java index dbaf7ba3..54cfff37 100644 --- a/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java +++ b/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java @@ -28,6 +28,7 @@ import com.linkedin.hoptimator.Deployer; import com.linkedin.hoptimator.PendingDelete; +import com.linkedin.hoptimator.Sink; import com.linkedin.hoptimator.Trigger; import com.linkedin.hoptimator.UserJob; import com.linkedin.hoptimator.Validated; @@ -517,6 +518,8 @@ private void deployImplicitTrigger(Map tierMap, Map tierMap, Map hoptimatorDriverMock; @Mock - MockedStatic validationServiceMock; + MockedStatic validationServiceMock; @Mock K8sLogicalTableDeployer mockCrdDeployer; @@ -144,7 +146,7 @@ K8sLogicalTableDeployer createLogicalTableDeployer( @Override void deployPipelineBundle(String fromTier, String toTier, - Map tierDatabases, + Map tierDatabases, Map tierSources, K8sContext ownerContext, boolean update) { // No-op in CRD-focused tests — pipeline deployment is tested separately. @@ -973,10 +975,10 @@ void implicitTriggerPathUsesOfflineTierPhysicalPath() throws Exception { // testSource() has path [logical, testevent]; offline tier's Database has schema=OFFLINE. // Expect the trigger to target the offline physical schema, not the logical one. - assertEquals("OFFLINE", capture.trigger.schema(), - "Trigger schema must point at the offline tier's physical schema"); - assertEquals("testevent", capture.trigger.table(), - "Trigger table must be the logical table name"); + assertEquals("OFFLINE", capture.trigger.source().schema(), + "Trigger source must point at the offline tier's physical schema"); + assertEquals("testevent", capture.trigger.source().table(), + "Trigger source table must be the logical table name"); } @Test @@ -1054,7 +1056,7 @@ void implicitTriggerForwardsSourceOptionsToTriggerOptions() throws Exception { doReturn(new V1OwnerReference()).when(mockCrdDeployer).createAndReference(); // CREATE TABLE WITH ("ab.cd" "customValue", "job.properties.online.name" "testevent-online") - Map sourceOptions = new java.util.HashMap<>(); + Map sourceOptions = new HashMap<>(); sourceOptions.put("ab.cd", "customValue"); sourceOptions.put("job.properties.online.name", "testevent-online"); Source src = new Source("logical", Arrays.asList("logical", "testevent"), sourceOptions); From 64dac5c73ace51a65dd27b60003825712560ffa1 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Tue, 12 May 2026 09:48:07 -0400 Subject: [PATCH 11/15] Extend pre-delete dependency guard to TableTrigger CRDs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Triggers carry the same depends-on- labels Pipelines do (stamped by K8sTriggerDeployer in the previous commit), but the dep-guard's PipelineDependencyChecker only consulted the Pipeline CRD list. That left a hole: a user could DROP TABLE on a source still referenced by a live trigger. PipelineDependencyChecker now runs the same label-selector + annotation-confirmation logic against TableTrigger CRDs as well. The inner loop is genericized over KubernetesObject; each blocker is tagged with its CRD kind (pipeline/foo, trigger/bar) so the error message points the user at what to unhook. Self-owner exclusion still applies — LogicalTable-owned triggers don't block their parent's cascade-delete. Coverage: - PipelineDependencyCheckerTest gets paired blocksOnExternalTrigger, skipsSelfOwnedTrigger, and errorMessageListsAllBlockersAcrossKinds cases that prove triggers participate alongside pipelines. - New k8s-trigger-validation.id integration scenario: CREATE TABLE → CREATE TRIGGER → DROP TABLE blocked → DROP TRIGGER → DROP TABLE succeeds. Mirrors the MV pattern the existing k8s-validation.id scenarios use. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../k8s/PipelineDependencyChecker.java | 89 ++++++++----- .../k8s/PipelineDependencyCheckerTest.java | 120 ++++++++++++++---- .../hoptimator/k8s/TestSqlScripts.java | 8 +- .../test/resources/k8s-trigger-validation.id | 32 +++++ 4 files changed, 186 insertions(+), 63 deletions(-) create mode 100644 hoptimator-k8s/src/test/resources/k8s-trigger-validation.id diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java index 8cbcb2d6..ba753d8e 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java @@ -5,29 +5,33 @@ import java.util.Collection; import java.util.List; +import io.kubernetes.client.common.KubernetesObject; import io.kubernetes.client.openapi.models.V1ObjectMeta; import io.kubernetes.client.openapi.models.V1OwnerReference; import com.linkedin.hoptimator.k8s.models.V1alpha1Pipeline; import com.linkedin.hoptimator.k8s.models.V1alpha1PipelineList; +import com.linkedin.hoptimator.k8s.models.V1alpha1TableTrigger; +import com.linkedin.hoptimator.k8s.models.V1alpha1TableTriggerList; import javax.annotation.Nullable; /** - * Checks whether any Pipeline CRDs still depend on a resource a {@link com.linkedin.hoptimator.Deployer} - * is about to delete. + * Checks whether any Pipeline or TableTrigger CRDs still depend on a resource a + * {@link com.linkedin.hoptimator.Deployer} is about to delete. * - *

    The lookup is a label-selector list against the Pipeline CRD group, so it is O(matches) on - * the wire — not a full scan. Each candidate is then cross-checked against the union of the - * {@link PipelineDependencyLabels#ANNOTATION_KEY_SOURCES sources} and - * {@link PipelineDependencyLabels#ANNOTATION_KEY_SINK sink} annotations to rule out hash - * collisions in the label slug and stale labels left over from a prior version of the pipeline's - * SQL ({@link K8sApi#update}'s additive label merge can leak old {@code depends-on-*} keys). + *

    Both CRDs carry the same {@code depends-on-} label and {@code depends-on-sources}/ + * {@code depends-on-sink} annotations (stamped by {@link K8sPipelineDeployer} and + * {@link K8sTriggerDeployer}), so the same lookup works for either: a label-selector list against + * the CRD group is O(matches) on the wire, then each candidate is cross-checked against the union + * of the source + sink annotations to rule out hash collisions in the slug and stale labels left + * over from a prior version of the resource ({@link K8sApi#update}'s additive label merge can leak + * old {@code depends-on-*} keys). * - *

    Pipelines owned (directly) by {@code (selfOwnerKind, selfOwnerName)} are excluded from the - * blocker list: those pipelines will be cascade-deleted alongside the parent resource, so counting - * them as external dependents would make composite deletes (e.g. {@code LogicalTableDeployer.delete()}) + *

    Resources owned (directly) by {@code (selfOwnerKind, selfOwnerName)} are excluded from the + * blocker list: those will be cascade-deleted alongside the parent resource, so counting them as + * external dependents would make composite deletes (e.g. {@code LogicalTableDeployer.delete()}) * impossible. */ public final class PipelineDependencyChecker { @@ -37,45 +41,64 @@ private PipelineDependencyChecker() { public static void assertNoExternalDependents(K8sContext context, String database, List path, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) throws SQLException { - assertNoExternalDependents(new K8sApi<>(context, K8sApiEndpoints.PIPELINES), + assertNoExternalDependents( + new K8sApi<>(context, K8sApiEndpoints.PIPELINES), + new K8sApi<>(context, K8sApiEndpoints.TABLE_TRIGGERS), database, path, selfOwnerKind, selfOwnerName); } - /** Variant that takes a pre-built {@link K8sApi} — used by tests to inject mocks. */ - static void assertNoExternalDependents(K8sApi api, + /** Variant that takes pre-built {@link K8sApi}s — used by tests to inject mocks. */ + static void assertNoExternalDependents(K8sApi pipelineApi, + K8sApi triggerApi, String database, List path, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) throws SQLException { String labelKey = PipelineDependencyLabels.labelKey(database, path); String identifier = PipelineDependencyLabels.identifier(database, path); - Collection matches = api.select(labelKey); + List blockers = new ArrayList<>(); + blockers.addAll(findBlockers(pipelineApi, labelKey, identifier, "pipeline", + selfOwnerKind, selfOwnerName)); + blockers.addAll(findBlockers(triggerApi, labelKey, identifier, "trigger", + selfOwnerKind, selfOwnerName)); + + if (!blockers.isEmpty()) { + throw new SQLException(String.format( + "Cannot delete %s — %d active dependent(s): %s", + identifier, blockers.size(), String.join(", ", blockers))); + } + } + /** + * Generic blocker enumeration: list resources of type {@code T} that carry the given + * {@code labelKey}, confirm via the depends-on annotations, and exclude self-owned children. + * The {@code kindLabel} is prefixed onto each blocker description so a unified error message + * can attribute each entry to its CRD kind. + */ + private static List findBlockers(K8sApi api, + String labelKey, String identifier, String kindLabel, + @Nullable String selfOwnerKind, @Nullable String selfOwnerName) throws SQLException { + Collection matches = api.select(labelKey); List blockers = new ArrayList<>(); - for (V1alpha1Pipeline p : matches) { - if (isSelfOwned(p, selfOwnerKind, selfOwnerName)) { + for (T obj : matches) { + V1ObjectMeta meta = obj.getMetadata(); + if (isSelfOwned(meta, selfOwnerKind, selfOwnerName)) { continue; } - if (!annotationConfirms(p, identifier)) { - // Label matched but annotation doesn't — this is a slug collision or a stale label, skip it. + if (!annotationConfirms(meta, identifier)) { + // Label matched but annotation doesn't — slug collision or stale label, skip it. continue; } - blockers.add(describeBlocker(p)); - } - - if (!blockers.isEmpty()) { - throw new SQLException(String.format( - "Cannot delete %s — %d active pipeline(s) depend on it: %s", - identifier, blockers.size(), String.join(", ", blockers))); + blockers.add(kindLabel + "/" + describeBlocker(meta)); } + return blockers; } - private static boolean isSelfOwned(V1alpha1Pipeline pipeline, @Nullable String selfOwnerKind, + private static boolean isSelfOwned(V1ObjectMeta meta, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) { if (selfOwnerKind == null || selfOwnerName == null) { return false; } - V1ObjectMeta meta = pipeline.getMetadata(); if (meta == null || meta.getOwnerReferences() == null) { return false; } @@ -87,10 +110,9 @@ private static boolean isSelfOwned(V1alpha1Pipeline pipeline, @Nullable String s return false; } - private static boolean annotationConfirms(V1alpha1Pipeline pipeline, String identifier) { - V1ObjectMeta meta = pipeline.getMetadata(); + private static boolean annotationConfirms(V1ObjectMeta meta, String identifier) { if (meta == null || meta.getAnnotations() == null) { - return true; // pre-labeling pipeline — conservatively trust the label match + return true; // pre-labeling resource — conservatively trust the label match } String sourcesAnno = meta.getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES); String sinkAnno = meta.getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SINK); @@ -104,11 +126,10 @@ private static boolean annotationConfirms(V1alpha1Pipeline pipeline, String iden } /** - * Builds a human-readable blocker description: the pipeline name, plus (when present) the top + * Builds a human-readable blocker description: the resource name, plus (when present) the top * ownerReference's {@code kind/name} so the user knows which higher-level resource owns it. */ - private static String describeBlocker(V1alpha1Pipeline pipeline) { - V1ObjectMeta meta = pipeline.getMetadata(); + private static String describeBlocker(V1ObjectMeta meta) { String name = meta == null ? "" : meta.getName(); String ownerSuffix = ""; if (meta != null && meta.getOwnerReferences() != null && !meta.getOwnerReferences().isEmpty()) { diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java index 6147686d..ca574f89 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java @@ -17,11 +17,14 @@ import com.linkedin.hoptimator.k8s.models.V1alpha1Pipeline; import com.linkedin.hoptimator.k8s.models.V1alpha1PipelineList; +import com.linkedin.hoptimator.k8s.models.V1alpha1TableTrigger; +import com.linkedin.hoptimator.k8s.models.V1alpha1TableTriggerList; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @@ -29,7 +32,9 @@ class PipelineDependencyCheckerTest { @Mock - private K8sApi api; + private K8sApi pipelineApi; + @Mock + private K8sApi triggerApi; private static final String DB = "kafka1"; private static final List PATH = Collections.singletonList("my-topic"); @@ -37,106 +42,165 @@ class PipelineDependencyCheckerTest { private static V1alpha1Pipeline pipeline(String name, String ownerKind, String ownerName, String annotationValue) { + return new V1alpha1Pipeline().metadata(metadata(name, ownerKind, ownerName, annotationValue)); + } + + private static V1alpha1TableTrigger trigger(String name, String ownerKind, String ownerName, + String sourcesAnnotation) { + return new V1alpha1TableTrigger().metadata(metadata(name, ownerKind, ownerName, sourcesAnnotation)); + } + + private static V1ObjectMeta metadata(String name, String ownerKind, String ownerName, + String annotationValue) { V1ObjectMeta meta = new V1ObjectMeta().name(name); if (ownerKind != null && ownerName != null) { meta.addOwnerReferencesItem(new V1OwnerReference().kind(ownerKind).name(ownerName)); } if (annotationValue != null) { - // Treat the test's single value as the sources annotation. The cross-check now reads - // sources ∪ sink, so writing to either one is sufficient to confirm the identifier. Map annotations = new HashMap<>(); annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, annotationValue); meta.setAnnotations(annotations); } - return new V1alpha1Pipeline().metadata(meta); + return meta; + } + + /** Default both APIs to empty so each test only stubs the side it cares about. */ + private void emptyByDefault() throws SQLException { + String labelKey = PipelineDependencyLabels.labelKey(DB, PATH); + lenient().when(pipelineApi.select(labelKey)).thenReturn(Collections.emptyList()); + lenient().when(triggerApi.select(labelKey)).thenReturn(Collections.emptyList()); } @Test - void passesWhenNoPipelinesMatch() throws SQLException { - when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Collections.emptyList()); + void passesWhenNoMatches() throws SQLException { + emptyByDefault(); - assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); + assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( + pipelineApi, triggerApi, DB, PATH, null, null)); } @Test void blocksOnExternalPipeline() throws SQLException { - when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + emptyByDefault(); + when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("ext-pipe", "View", "owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); + () -> PipelineDependencyChecker.assertNoExternalDependents( + pipelineApi, triggerApi, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("ext-pipe")); assertTrue(ex.getMessage().contains(IDENTIFIER)); + assertTrue(ex.getMessage().contains("pipeline/ext-pipe"), + "blocker should be tagged with its CRD kind: " + ex.getMessage()); + } + + @Test + void blocksOnExternalTrigger() throws SQLException { + emptyByDefault(); + when(triggerApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + .thenReturn(Collections.singletonList(trigger("backfill", "LogicalTable", "lt-name", IDENTIFIER))); + + SQLException ex = assertThrows(SQLException.class, + () -> PipelineDependencyChecker.assertNoExternalDependents( + pipelineApi, triggerApi, DB, PATH, null, null)); + assertTrue(ex.getMessage().contains("trigger/backfill"), + "blocker should be tagged with its CRD kind so the user knows what to look at: " + + ex.getMessage()); + assertTrue(ex.getMessage().contains("LogicalTable/lt-name"), + "owner reference should appear in the blocker description: " + ex.getMessage()); } @Test void skipsSelfOwnedPipeline() throws SQLException { - when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + emptyByDefault(); + when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("owned-pipe", "LogicalTable", "self-name", IDENTIFIER))); assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( - api, DB, PATH, "LogicalTable", "self-name")); + pipelineApi, triggerApi, DB, PATH, "LogicalTable", "self-name")); + } + + @Test + void skipsSelfOwnedTrigger() throws SQLException { + emptyByDefault(); + when(triggerApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + .thenReturn(Collections.singletonList(trigger("owned-trigger", "LogicalTable", "self-name", IDENTIFIER))); + + // Triggers spawned by the same LogicalTable cascade-delete with the parent — they must not + // count as external dependents or you can't drop the LT at all. + assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( + pipelineApi, triggerApi, DB, PATH, "LogicalTable", "self-name")); } @Test void blocksOnExternalWhenSomeAreSelfOwned() throws SQLException { - when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( + emptyByDefault(); + when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( pipeline("owned-pipe", "LogicalTable", "self-name", IDENTIFIER), pipeline("external-pipe", "View", "other-owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, () -> PipelineDependencyChecker.assertNoExternalDependents( - api, DB, PATH, "LogicalTable", "self-name")); + pipelineApi, triggerApi, DB, PATH, "LogicalTable", "self-name")); assertTrue(ex.getMessage().contains("external-pipe")); assertFalse(ex.getMessage().contains("owned-pipe"), "self-owned pipeline must not be listed"); } @Test void rejectsSlugCollisionViaAnnotation() throws SQLException { + emptyByDefault(); // Pipeline labels collide on the slug (which is what api.select matched on) but the // annotation reveals the actual identifier is different — so this should NOT block. - when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("colliding-pipe", "View", "owner", "some-other-database/some-other-path"))); - assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); + assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( + pipelineApi, triggerApi, DB, PATH, null, null)); } @Test void treatsMissingAnnotationAsTrusted() throws SQLException { + emptyByDefault(); // A pipeline with the matching label but no depends-on annotation (pre-labeling migration // case, or future code path that didn't write the annotation) is still treated as a blocker. - when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("legacy-pipe", "View", "owner", null))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); + () -> PipelineDependencyChecker.assertNoExternalDependents( + pipelineApi, triggerApi, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("legacy-pipe")); } @Test void errorMessageIncludesOwnerKindAndName() throws SQLException { - when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))) + emptyByDefault(); + when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("ext-pipe", "View", "owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); + () -> PipelineDependencyChecker.assertNoExternalDependents( + pipelineApi, triggerApi, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("View/owner"), "error should name the owning View so the user knows what to unhook: " + ex.getMessage()); } @Test - void errorMessageListsAllBlockers() throws SQLException { - when(api.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( + void errorMessageListsAllBlockersAcrossKinds() throws SQLException { + emptyByDefault(); + when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( pipeline("p1", "View", "owner1", IDENTIFIER), - pipeline("p2", "View", "owner2", IDENTIFIER), - pipeline("p3", "View", "owner3", IDENTIFIER))); + pipeline("p2", "View", "owner2", IDENTIFIER))); + when(triggerApi.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn( + Collections.singletonList(trigger("t1", "LogicalTable", "lt-name", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents(api, DB, PATH, null, null)); - assertTrue(ex.getMessage().contains("p1")); - assertTrue(ex.getMessage().contains("p2")); - assertTrue(ex.getMessage().contains("p3")); - assertTrue(ex.getMessage().contains("3 active pipeline")); + () -> PipelineDependencyChecker.assertNoExternalDependents( + pipelineApi, triggerApi, DB, PATH, null, null)); + assertTrue(ex.getMessage().contains("pipeline/p1")); + assertTrue(ex.getMessage().contains("pipeline/p2")); + assertTrue(ex.getMessage().contains("trigger/t1")); + assertTrue(ex.getMessage().contains("3 active dependent")); } } diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java index 4114448e..b8a61406 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/TestSqlScripts.java @@ -1,9 +1,10 @@ package com.linkedin.hoptimator.k8s; -import com.linkedin.hoptimator.jdbc.QuidemTestBase; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import com.linkedin.hoptimator.jdbc.QuidemTestBase; + @Tag("integration") public class TestSqlScripts extends QuidemTestBase { @@ -43,6 +44,11 @@ public void k8sTriggerOptions() throws Exception { run("k8s-trigger-options.id"); } + @Test + public void k8sTriggerValidation() throws Exception { + run("k8s-trigger-validation.id"); + } + @Test public void k8sDdlCreateDatabase() throws Exception { run("k8s-ddl-create-database.id"); diff --git a/hoptimator-k8s/src/test/resources/k8s-trigger-validation.id b/hoptimator-k8s/src/test/resources/k8s-trigger-validation.id new file mode 100644 index 00000000..24993f41 --- /dev/null +++ b/hoptimator-k8s/src/test/resources/k8s-trigger-validation.id @@ -0,0 +1,32 @@ +!set outputformat mysql +!use k8s + +# Pre-delete dependency guard for triggers. A trigger carries the same +# `depends-on-` label that pipelines do (stamped from `create trigger`'s +# target identifier), so dropping a source still referenced by a live trigger +# is blocked with the same "active dependent(s)" message a pipeline would +# trigger. Drops the trigger to clear the guard, then re-drops the source. + +create or replace table ads."triggertable" ("i" int, "s" VARCHAR); +(0 rows modified) + +!update + +create trigger blocks_drop on ads."triggertable" as 'my-app' in 'my-mp' with (key 'value'); +(0 rows modified) + +!update + +drop table ads."triggertable"; +active dependent +!error + +drop trigger blocks_drop; +(0 rows modified) + +!update + +drop table ads."triggertable"; +(0 rows modified) + +!update From a158eda251bbc5a42284785b1ef6cce9e2a4239a Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Tue, 12 May 2026 12:15:19 -0400 Subject: [PATCH 12/15] Generalize dependency labels/checker beyond pipelines MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit PipelineDependency{Labels,Checker} and K8sPipelineDependencyValidator were specific to Pipeline CRDs in name only — TableTrigger CRDs now wear the same depends-on labels and annotations. Renamed to DependencyLabels, DependencyChecker, and K8sDependencyValidator. Collapsed the labels API: DependencyLabels now exposes a single stamp(V1ObjectMeta, Collection, Sink) that writes both the depends-on labels and the directional annotations. K8sTriggerDeployer had reimplemented this inline in two places; both call sites now collapse to one stamp() call. Also dropped the 5-arg Trigger constructor; callers must pass a sink (or null) explicitly. --- .../java/com/linkedin/hoptimator/Trigger.java | 17 +- .../hoptimator/PendingDeleteTest.java | 2 +- .../com/linkedin/hoptimator/TriggerTest.java | 8 +- .../jdbc/HoptimatorDdlExecutor.java | 13 +- ...ncyChecker.java => DependencyChecker.java} | 14 +- ...dencyLabels.java => DependencyLabels.java} | 72 +++---- ...dator.java => K8sDependencyValidator.java} | 8 +- .../hoptimator/k8s/K8sPipelineBundle.java | 2 +- .../hoptimator/k8s/K8sPipelineDeployer.java | 19 +- .../hoptimator/k8s/K8sTriggerDeployer.java | 51 ++--- .../hoptimator/k8s/K8sValidatorProvider.java | 2 +- ...erTest.java => DependencyCheckerTest.java} | 46 ++--- .../hoptimator/k8s/DependencyLabelsTest.java | 178 ++++++++++++++++++ ...t.java => K8sDependencyValidatorTest.java} | 28 +-- .../k8s/K8sPipelineDeployerTest.java | 10 +- .../k8s/K8sTriggerDeployerTest.java | 56 +++--- .../k8s/K8sValidatorProviderTest.java | 4 +- .../k8s/PipelineDependencyLabelsTest.java | 163 ---------------- 18 files changed, 330 insertions(+), 363 deletions(-) rename hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/{PipelineDependencyChecker.java => DependencyChecker.java} (91%) rename hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/{PipelineDependencyLabels.java => DependencyLabels.java} (66%) rename hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/{K8sPipelineDependencyValidator.java => K8sDependencyValidator.java} (74%) rename hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/{PipelineDependencyCheckerTest.java => DependencyCheckerTest.java} (81%) create mode 100644 hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyLabelsTest.java rename hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/{K8sPipelineDependencyValidatorTest.java => K8sDependencyValidatorTest.java} (78%) delete mode 100644 hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java diff --git a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Trigger.java b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Trigger.java index 1aefdbb7..a24d4b91 100644 --- a/hoptimator-api/src/main/java/com/linkedin/hoptimator/Trigger.java +++ b/hoptimator-api/src/main/java/com/linkedin/hoptimator/Trigger.java @@ -1,5 +1,6 @@ package com.linkedin.hoptimator; +import javax.annotation.Nullable; import java.util.Map; @@ -14,19 +15,13 @@ public class Trigger implements Deployable { private final Source source; private final Sink sink; - public Trigger(String name, UserJob job, String cronSchedule, Map options, - Source source) { - this(name, job, cronSchedule, options, source, null); - } - /** - * Variant accepting an optional downstream sink. Set by {@code LogicalTableDeployer} on - * bridging-tier triggers so the visualizer can render the directional flow - * {@code source -.-> trigger -.-> sink}, and so the dep-guard finds the sink-side - * dependency. {@code null} for user-created triggers (no declared sink). + * Contains an optional downstream sink for triggers that operate between a source + * sink (think ETL/rETL). + * TODO: need to collapse the "job.properties.online.table.name" logic into a sink for adhoc triggers */ public Trigger(String name, UserJob job, String cronSchedule, Map options, - Source source, Sink sink) { + Source source, @Nullable Sink sink) { this.name = name; this.job = job; this.cronSchedule = cronSchedule; @@ -58,7 +53,7 @@ public Source source() { } /** Downstream sink the trigger's job writes to, or {@code null} when the trigger has no declared sink. */ - public Sink sink() { + public @Nullable Sink sink() { return sink; } diff --git a/hoptimator-api/src/test/java/com/linkedin/hoptimator/PendingDeleteTest.java b/hoptimator-api/src/test/java/com/linkedin/hoptimator/PendingDeleteTest.java index 4b5b9144..99abc023 100644 --- a/hoptimator-api/src/test/java/com/linkedin/hoptimator/PendingDeleteTest.java +++ b/hoptimator-api/src/test/java/com/linkedin/hoptimator/PendingDeleteTest.java @@ -67,7 +67,7 @@ void toStringOmitsSelfOwnerWhenNull() { @Test void toStringOmitsSelfOwnerWhenOnlyOneFieldSet() { // The toString contract says self= appears only when both kind and name are non-null. - // (The K8sPipelineDependencyChecker.isSelfOwned guard also requires both.) + // (The DependencyChecker.isSelfOwned guard also requires both.) PendingDelete kindOnly = new PendingDelete<>("t", "LogicalTable", null); assertFalse(kindOnly.toString().contains("self=")); diff --git a/hoptimator-api/src/test/java/com/linkedin/hoptimator/TriggerTest.java b/hoptimator-api/src/test/java/com/linkedin/hoptimator/TriggerTest.java index 2dbec09e..4f901499 100644 --- a/hoptimator-api/src/test/java/com/linkedin/hoptimator/TriggerTest.java +++ b/hoptimator-api/src/test/java/com/linkedin/hoptimator/TriggerTest.java @@ -18,7 +18,7 @@ void testAccessors() { UserJob userJob = new UserJob("ns", "jobName"); Map options = Map.of("paused", "true"); Source source = new Source("db", List.of("schema", "table"), Collections.emptyMap()); - Trigger trigger = new Trigger("myTrigger", userJob, "0 * * * *", options, source); + Trigger trigger = new Trigger("myTrigger", userJob, "0 * * * *", options, source, null); assertEquals("myTrigger", trigger.name()); assertEquals(userJob, trigger.job()); @@ -33,7 +33,7 @@ void testAccessors() { @Test void testNullSourceAccessor() { - Trigger trigger = new Trigger("t", null, null, Map.of(), null); + Trigger trigger = new Trigger("t", null, null, Map.of(), null, null); assertNull(trigger.source()); assertNull(trigger.sink()); } @@ -41,13 +41,13 @@ void testNullSourceAccessor() { @Test void testToString() { Source source = new Source("db", List.of("a", "b"), Collections.emptyMap()); - Trigger trigger = new Trigger("myTrig", null, null, Map.of(), source); + Trigger trigger = new Trigger("myTrig", null, null, Map.of(), source, null); assertEquals("Trigger[myTrig, a.b]", trigger.toString()); } @Test void testToStringWithoutSource() { - Trigger trigger = new Trigger("myTrig", null, null, Map.of(), null); + Trigger trigger = new Trigger("myTrig", null, null, Map.of(), null, null); assertEquals("Trigger[myTrig, ]", trigger.toString()); } } diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index d87f8ef5..8ce3f938 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -233,7 +233,7 @@ public void execute(SqlCreateTrigger create, CalcitePrepare.Context context) { // stamp depends-on labels: same identifier shape as Pipelines, so the dep-guard finds the // trigger when somebody tries to drop the source it points at. Source source = new Source(databaseOf(target), targetPath, Collections.emptyMap()); - Trigger trigger = new Trigger(name, job, cronSchedule, options, source); + Trigger trigger = new Trigger(name, job, cronSchedule, options, source, null); Collection deployers = null; try { @@ -261,11 +261,8 @@ public void execute(SqlCreateTrigger create, CalcitePrepare.Context context) { /** * Best-effort lookup of the database name that owns a Calcite Table. Mirrors what the - * {@code DROP TABLE} path does (lines 432–437) so triggers and tables agree on the same - * {@code (database, path)} identifier, which is what the dep-guard's depends-on labels are - * keyed on. Returns {@code null} for table types that don't carry a database (e.g. ad-hoc - * Calcite views) — depends-on labels are skipped in that case, so the trigger simply won't - * participate in the dep-guard. + * {@code DROP TABLE} path does so triggers and tables agree on the same + * {@code (database, path)} identifier. */ private static String databaseOf(Table target) { if (target instanceof HoptimatorJdbcTable) { @@ -331,7 +328,7 @@ public void execute(SqlDropTrigger drop, CalcitePrepare.Context context) { } String name = drop.name.names.get(0); - Trigger trigger = new Trigger(name, null, null, new HashMap<>(), null); + Trigger trigger = new Trigger(name, null, null, new HashMap<>(), null, null); Collection deployers = null; try { @@ -368,7 +365,7 @@ private void updateTriggerPausedState(SqlNode sqlNode, SqlIdentifier triggerName Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, String.valueOf(paused)); - Trigger trigger = new Trigger(name, null, null, options, null); + Trigger trigger = new Trigger(name, null, null, options, null, null); Collection deployers = null; try { diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyChecker.java similarity index 91% rename from hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java rename to hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyChecker.java index ba753d8e..cd2ecf8c 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyChecker.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyChecker.java @@ -34,9 +34,9 @@ * external dependents would make composite deletes (e.g. {@code LogicalTableDeployer.delete()}) * impossible. */ -public final class PipelineDependencyChecker { +public final class DependencyChecker { - private PipelineDependencyChecker() { + private DependencyChecker() { } public static void assertNoExternalDependents(K8sContext context, String database, @@ -53,8 +53,8 @@ static void assertNoExternalDependents(K8sApi path, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) throws SQLException { - String labelKey = PipelineDependencyLabels.labelKey(database, path); - String identifier = PipelineDependencyLabels.identifier(database, path); + String labelKey = DependencyLabels.labelKey(database, path); + String identifier = DependencyLabels.identifier(database, path); List blockers = new ArrayList<>(); blockers.addAll(findBlockers(pipelineApi, labelKey, identifier, "pipeline", @@ -114,12 +114,12 @@ private static boolean annotationConfirms(V1ObjectMeta meta, String identifier) if (meta == null || meta.getAnnotations() == null) { return true; // pre-labeling resource — conservatively trust the label match } - String sourcesAnno = meta.getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES); - String sinkAnno = meta.getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SINK); + String sourcesAnno = meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SOURCES); + String sinkAnno = meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINK); if (sourcesAnno == null && sinkAnno == null) { return true; // same — no annotations to cross-check against } - if (sourcesAnno != null && PipelineDependencyLabels.parseAnnotation(sourcesAnno).contains(identifier)) { + if (sourcesAnno != null && DependencyLabels.parseAnnotation(sourcesAnno).contains(identifier)) { return true; } return identifier.equals(sinkAnno); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyLabels.java similarity index 66% rename from hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java rename to hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyLabels.java index 07db426e..74ee3f77 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabels.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyLabels.java @@ -4,23 +4,29 @@ import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.Collection; -import java.util.LinkedHashMap; +import java.util.HashMap; import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; +import javax.annotation.Nullable; + +import io.kubernetes.client.openapi.models.V1ObjectMeta; + import com.linkedin.hoptimator.Sink; import com.linkedin.hoptimator.Source; /** - * Computes the labels and annotations that encode a Pipeline CRD's dependency edges. + * Encodes a resource's dependency edges as K8s labels + annotations. Used by both + * {@link K8sPipelineDeployer} and {@link K8sTriggerDeployer} so the dep-guard and + * the visualizer can find dependents with a single label-selector query. * - *

    Every source and sink a pipeline references is recorded as a label: + *

    Every source and sink the resource references becomes a label: * {@code hoptimator.linkedin.com/depends-on-: "_"} where * {@code } is a deterministic hash derived from {@code database + "_" + pathString}. * The hash keeps label keys within Kubernetes's 63-character name limit for arbitrary paths, - * and lets {@code K8sApi.select} filter pipelines by dependency on the server. + * and lets {@code K8sApi.select} filter by dependency on the server. * *

    Two annotations preserve the directional information the labels lose: *

      @@ -32,11 +38,11 @@ *
    • Collision guard for the delete-time dependency checker — a label match is only * trusted when the resource appears in either annotation, so slug collisions and stale * labels (from {@link K8sApi#update}'s additive label merge) can't produce false positives.
    • - *
    • Direction recovery for visualization — the renderer draws source → pipeline → sink + *
    • Direction recovery for visualization — the renderer draws source → resource → sink * arrows from the split.
    • * */ -public final class PipelineDependencyLabels { +public final class DependencyLabels { static final String LABEL_PREFIX = "hoptimator.linkedin.com/depends-on-"; /** Annotation listing only source identifiers. */ @@ -47,7 +53,7 @@ public final class PipelineDependencyLabels { private static final int SLUG_LENGTH = 16; // 64 bits of SHA-256 → ~1 in 1.8e19 collisions private static final int MAX_LABEL_VALUE = 63; - private PipelineDependencyLabels() { + private DependencyLabels() { } /** @@ -67,46 +73,40 @@ public static String slug(String database, Iterable path) { return sb.toString(); } - /** Label key a Pipeline carries if it depends on the given resource. */ + /** Label key a resource carries if it depends on the given source/sink. */ public static String labelKey(String database, Iterable path) { return LABEL_PREFIX + slug(database, path); } /** - * Labels to stamp on a Pipeline CRD — one entry per source and the sink. Both edges - * matter to the guard: dropping a source orphans pipelines that read from it; dropping a sink - * orphans pipelines that write to it. + * Stamps the depends-on labels and directional annotations onto {@code meta}, merging with any + * existing labels/annotations on the object. Both edges matter to the guard: dropping a source + * orphans resources that read from it; dropping a sink orphans resources that write to it. * - *

      Keys are the same as {@link #labelKey}. Values are the readable identifier, truncated - * to 63 chars if necessary (the annotation preserves the untruncated form). Values are - * for debugging purposes only. + *

      Sources whose {@code database()} is null are skipped — they don't have a stable identifier. */ - public static Map labelsFor(Collection sources, Sink sink) { - Map labels = new LinkedHashMap<>(); + public static void stamp(V1ObjectMeta meta, Collection sources, @Nullable Sink sink) { + Map labels = meta.getLabels() != null ? meta.getLabels() : new HashMap<>(); + Map annotations = meta.getAnnotations() != null ? meta.getAnnotations() : new HashMap<>(); + Set sourceIds = new LinkedHashSet<>(); for (Source src : sources) { - labels.put(labelKey(src.database(), src.path()), truncate(identifier(src.database(), src.path()))); + if (src == null || src.database() == null) { + continue; + } + String id = identifier(src.database(), src.path()); + labels.put(labelKey(src.database(), src.path()), truncate(id)); + sourceIds.add(id); } - if (sink != null) { - labels.put(labelKey(sink.database(), sink.path()), truncate(identifier(sink.database(), sink.path()))); + if (!sourceIds.isEmpty()) { + annotations.put(ANNOTATION_KEY_SOURCES, String.join(",", sourceIds)); } - return labels; - } - - /** Comma-separated list of source identifiers (no sink). Empty string if no sources. */ - public static String sourcesAnnotation(Collection sources) { - Set ids = new LinkedHashSet<>(); - for (Source src : sources) { - ids.add(identifier(src.database(), src.path())); - } - return String.join(",", ids); - } - - /** Single sink identifier, or {@code null} if there is no sink. */ - public static String sinkAnnotation(Sink sink) { - if (sink == null) { - return null; + if (sink != null && sink.database() != null) { + String id = identifier(sink.database(), sink.path()); + labels.put(labelKey(sink.database(), sink.path()), truncate(id)); + annotations.put(ANNOTATION_KEY_SINK, id); } - return identifier(sink.database(), sink.path()); + meta.setLabels(labels); + meta.setAnnotations(annotations); } /** Parses the collision-guard annotation back into the set of identifiers it encoded. */ diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sDependencyValidator.java similarity index 74% rename from hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java rename to hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sDependencyValidator.java index ff06a3be..5a746206 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidator.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sDependencyValidator.java @@ -11,16 +11,16 @@ /** * Pre-delete dependency check, run by the validator framework when a {@link Source} is wrapped * in {@link com.linkedin.hoptimator.PendingDelete}. Delegates to the existing - * {@link PipelineDependencyChecker} (label-selector + annotation collision-guard + self-owner + * {@link DependencyChecker} (label-selector + annotation collision-guard + self-owner * exclusion) and surfaces any blocking pipeline as a validation error. */ -final class K8sPipelineDependencyValidator implements Validator { +final class K8sDependencyValidator implements Validator { private final Source source; private final String selfOwnerKind; private final String selfOwnerName; - K8sPipelineDependencyValidator(Source source, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) { + K8sDependencyValidator(Source source, @Nullable String selfOwnerKind, @Nullable String selfOwnerName) { this.source = source; this.selfOwnerKind = selfOwnerKind; this.selfOwnerName = selfOwnerName; @@ -29,7 +29,7 @@ final class K8sPipelineDependencyValidator implements Validator { @Override public void validate(Issues issues, Connection connection) { try { - PipelineDependencyChecker.assertNoExternalDependents( + DependencyChecker.assertNoExternalDependents( K8sContext.create(connection), source.database(), source.path(), selfOwnerKind, selfOwnerName); } catch (SQLException e) { diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineBundle.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineBundle.java index 8fe455d2..5a09a6da 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineBundle.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineBundle.java @@ -30,7 +30,7 @@ public class K8sPipelineBundle implements Deployer { /** * {@code sources} and {@code sink} are stamped as {@code depends-on-*} - * labels on the Pipeline CRD so the delete-time guard in {@link PipelineDependencyChecker} + * labels on the Pipeline CRD so the delete-time guard in {@link DependencyChecker} * can find this pipeline by label selector. */ public K8sPipelineBundle(String name, List pipelineSpecs, String sql, diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java index 551dfee3..f9f7e840 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java @@ -3,9 +3,7 @@ import java.sql.SQLException; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import io.kubernetes.client.openapi.models.V1ObjectMeta; @@ -19,7 +17,7 @@ /** * Deploys a Pipeline object. Stamps {@code depends-on-*} labels and a {@code depends-on} * collision-guard annotation describing which sources/sink the pipeline references, so - * {@link PipelineDependencyChecker} can look up dependents by label selector at delete time. + * {@link DependencyChecker} can look up dependents by label selector at delete time. * *

      {@link K8sApi#update} merges labels additively, so stale {@code depends-on-*} labels from * a previous version of the pipeline's SQL can linger. Correctness is preserved by the @@ -48,20 +46,7 @@ class K8sPipelineDeployer extends K8sDeployer labels = PipelineDependencyLabels.labelsFor(sources, sink); - if (!labels.isEmpty()) { - meta.setLabels(new HashMap<>(labels)); - Map annotations = new HashMap<>(); - String sourcesAnno = PipelineDependencyLabels.sourcesAnnotation(sources); - if (!sourcesAnno.isEmpty()) { - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, sourcesAnno); - } - String sinkAnno = PipelineDependencyLabels.sinkAnnotation(sink); - if (sinkAnno != null) { - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SINK, sinkAnno); - } - meta.setAnnotations(annotations); - } + DependencyLabels.stamp(meta, sources, sink); return new V1alpha1Pipeline().kind(K8sApiEndpoints.PIPELINES.kind()) .apiVersion(K8sApiEndpoints.PIPELINES.apiVersion()) .metadata(meta) diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java index aabdd276..2f9bf44f 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java @@ -1,6 +1,5 @@ package com.linkedin.hoptimator.k8s; -import com.linkedin.hoptimator.Sink; import com.linkedin.hoptimator.Source; import com.linkedin.hoptimator.Trigger; import com.linkedin.hoptimator.k8s.models.V1alpha1JobTemplate; @@ -12,6 +11,7 @@ import io.kubernetes.client.openapi.models.V1ObjectMeta; import java.sql.SQLException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -81,28 +81,15 @@ private void stampDependencyMetadata(V1alpha1TableTrigger target) { meta = new V1ObjectMeta(); target.metadata(meta); } - Map labels = meta.getLabels() != null ? meta.getLabels() : new HashMap<>(); - Map annotations = meta.getAnnotations() != null ? meta.getAnnotations() : new HashMap<>(); Source source = trigger.source(); - if (source != null && source.database() != null) { - String labelKey = PipelineDependencyLabels.labelKey(source.database(), source.path()); - String identifier = PipelineDependencyLabels.identifier(source.database(), source.path()); - labels.put(labelKey, identifier.length() <= 63 ? identifier : identifier.substring(0, 63)); - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, identifier); - } - Sink sink = trigger.sink(); - if (sink != null) { - String sinkLabelKey = PipelineDependencyLabels.labelKey(sink.database(), sink.path()); - String sinkIdentifier = PipelineDependencyLabels.identifier(sink.database(), sink.path()); - labels.put(sinkLabelKey, - sinkIdentifier.length() <= 63 ? sinkIdentifier : sinkIdentifier.substring(0, 63)); - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SINK, sinkIdentifier); - } + DependencyLabels.stamp(meta, + source != null ? Collections.singletonList(source) : Collections.emptyList(), + trigger.sink()); if (trigger.job() != null) { + Map annotations = meta.getAnnotations() != null ? meta.getAnnotations() : new HashMap<>(); annotations.put(JOB_TEMPLATE_ANNOTATION, trigger.job().name()); + meta.setAnnotations(annotations); } - meta.setLabels(labels); - meta.setAnnotations(annotations); } @Override @@ -135,28 +122,20 @@ protected V1alpha1TableTrigger toK8sObject() throws SQLException { .with("schema", source != null ? source.schema() : null) .with(properties); V1alpha1JobTemplate jobTemplate = jobTemplateApi.get(jobNamespace, jobName); + V1ObjectMeta meta = new V1ObjectMeta().name(triggerName); Map labels = new HashMap<>(); labels.put("view", viewName); // a corresponding view object may or may not exist. - Map annotations = new HashMap<>(); + meta.setLabels(labels); // Stamp depends-on labels so the visualizer (and the dep-guard) can find triggers via // label selector. The Trigger's source is the upstream table the job reads from. When the // trigger carries a Sink (set by LogicalTableDeployer for bridging triggers), we // additionally stamp the sink — that's what makes reverse-ETL flows render as connectors. - if (source != null && source.database() != null) { - String labelKey = PipelineDependencyLabels.labelKey(source.database(), source.path()); - String identifier = PipelineDependencyLabels.identifier(source.database(), source.path()); - labels.put(labelKey, identifier.length() <= 63 ? identifier : identifier.substring(0, 63)); - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, identifier); - } - Sink sink = trigger.sink(); - if (sink != null) { - String sinkLabelKey = PipelineDependencyLabels.labelKey(sink.database(), sink.path()); - String sinkIdentifier = PipelineDependencyLabels.identifier(sink.database(), sink.path()); - labels.put(sinkLabelKey, - sinkIdentifier.length() <= 63 ? sinkIdentifier : sinkIdentifier.substring(0, 63)); - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SINK, sinkIdentifier); - } + DependencyLabels.stamp(meta, + source != null ? Collections.singletonList(source) : Collections.emptyList(), + trigger.sink()); + Map annotations = meta.getAnnotations() != null ? meta.getAnnotations() : new HashMap<>(); annotations.put(JOB_TEMPLATE_ANNOTATION, trigger.job().name()); + meta.setAnnotations(annotations); String template = jobTemplate.getSpec().getYaml(); String rendered = new Template.SimpleTemplate(template).render(env); Map jobProps = new HashMap<>(); @@ -176,10 +155,6 @@ protected V1alpha1TableTrigger toK8sObject() throws SQLException { if (trigger.options().containsKey(Trigger.PAUSED_OPTION)) { spec.paused("true".equals(trigger.options().get(Trigger.PAUSED_OPTION))); } - V1ObjectMeta meta = new V1ObjectMeta().name(triggerName).labels(labels); - if (annotations != null) { - meta.setAnnotations(annotations); - } return new V1alpha1TableTrigger() .kind(K8sApiEndpoints.TABLE_TRIGGERS.kind()) .apiVersion(K8sApiEndpoints.TABLE_TRIGGERS.apiVersion()) diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java index 3cc45e4b..7cf14d20 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sValidatorProvider.java @@ -26,7 +26,7 @@ public Collection validators(T obj, Connection connection) { Object target = pd.target(); if (target instanceof Source) { return Collections.singletonList( - new K8sPipelineDependencyValidator((Source) target, pd.selfOwnerKind(), pd.selfOwnerName())); + new K8sDependencyValidator((Source) target, pd.selfOwnerKind(), pd.selfOwnerName())); } } return Collections.emptyList(); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyCheckerTest.java similarity index 81% rename from hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java rename to hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyCheckerTest.java index ca574f89..535c3987 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyCheckerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyCheckerTest.java @@ -29,7 +29,7 @@ @ExtendWith(MockitoExtension.class) -class PipelineDependencyCheckerTest { +class DependencyCheckerTest { @Mock private K8sApi pipelineApi; @@ -58,7 +58,7 @@ private static V1ObjectMeta metadata(String name, String ownerKind, String owner } if (annotationValue != null) { Map annotations = new HashMap<>(); - annotations.put(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES, annotationValue); + annotations.put(DependencyLabels.ANNOTATION_KEY_SOURCES, annotationValue); meta.setAnnotations(annotations); } return meta; @@ -66,7 +66,7 @@ private static V1ObjectMeta metadata(String name, String ownerKind, String owner /** Default both APIs to empty so each test only stubs the side it cares about. */ private void emptyByDefault() throws SQLException { - String labelKey = PipelineDependencyLabels.labelKey(DB, PATH); + String labelKey = DependencyLabels.labelKey(DB, PATH); lenient().when(pipelineApi.select(labelKey)).thenReturn(Collections.emptyList()); lenient().when(triggerApi.select(labelKey)).thenReturn(Collections.emptyList()); } @@ -75,18 +75,18 @@ private void emptyByDefault() throws SQLException { void passesWhenNoMatches() throws SQLException { emptyByDefault(); - assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( + assertDoesNotThrow(() -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, null, null)); } @Test void blocksOnExternalPipeline() throws SQLException { emptyByDefault(); - when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(pipelineApi.select(DependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("ext-pipe", "View", "owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents( + () -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("ext-pipe")); assertTrue(ex.getMessage().contains(IDENTIFIER)); @@ -97,11 +97,11 @@ void blocksOnExternalPipeline() throws SQLException { @Test void blocksOnExternalTrigger() throws SQLException { emptyByDefault(); - when(triggerApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(triggerApi.select(DependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(trigger("backfill", "LogicalTable", "lt-name", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents( + () -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("trigger/backfill"), "blocker should be tagged with its CRD kind so the user knows what to look at: " @@ -113,34 +113,34 @@ void blocksOnExternalTrigger() throws SQLException { @Test void skipsSelfOwnedPipeline() throws SQLException { emptyByDefault(); - when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(pipelineApi.select(DependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("owned-pipe", "LogicalTable", "self-name", IDENTIFIER))); - assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( + assertDoesNotThrow(() -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, "LogicalTable", "self-name")); } @Test void skipsSelfOwnedTrigger() throws SQLException { emptyByDefault(); - when(triggerApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(triggerApi.select(DependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(trigger("owned-trigger", "LogicalTable", "self-name", IDENTIFIER))); // Triggers spawned by the same LogicalTable cascade-delete with the parent — they must not // count as external dependents or you can't drop the LT at all. - assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( + assertDoesNotThrow(() -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, "LogicalTable", "self-name")); } @Test void blocksOnExternalWhenSomeAreSelfOwned() throws SQLException { emptyByDefault(); - when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( + when(pipelineApi.select(DependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( pipeline("owned-pipe", "LogicalTable", "self-name", IDENTIFIER), pipeline("external-pipe", "View", "other-owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents( + () -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, "LogicalTable", "self-name")); assertTrue(ex.getMessage().contains("external-pipe")); assertFalse(ex.getMessage().contains("owned-pipe"), "self-owned pipeline must not be listed"); @@ -151,11 +151,11 @@ void rejectsSlugCollisionViaAnnotation() throws SQLException { emptyByDefault(); // Pipeline labels collide on the slug (which is what api.select matched on) but the // annotation reveals the actual identifier is different — so this should NOT block. - when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(pipelineApi.select(DependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("colliding-pipe", "View", "owner", "some-other-database/some-other-path"))); - assertDoesNotThrow(() -> PipelineDependencyChecker.assertNoExternalDependents( + assertDoesNotThrow(() -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, null, null)); } @@ -164,11 +164,11 @@ void treatsMissingAnnotationAsTrusted() throws SQLException { emptyByDefault(); // A pipeline with the matching label but no depends-on annotation (pre-labeling migration // case, or future code path that didn't write the annotation) is still treated as a blocker. - when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(pipelineApi.select(DependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("legacy-pipe", "View", "owner", null))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents( + () -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("legacy-pipe")); } @@ -176,11 +176,11 @@ void treatsMissingAnnotationAsTrusted() throws SQLException { @Test void errorMessageIncludesOwnerKindAndName() throws SQLException { emptyByDefault(); - when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))) + when(pipelineApi.select(DependencyLabels.labelKey(DB, PATH))) .thenReturn(Collections.singletonList(pipeline("ext-pipe", "View", "owner", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents( + () -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("View/owner"), "error should name the owning View so the user knows what to unhook: " + ex.getMessage()); @@ -189,14 +189,14 @@ void errorMessageIncludesOwnerKindAndName() throws SQLException { @Test void errorMessageListsAllBlockersAcrossKinds() throws SQLException { emptyByDefault(); - when(pipelineApi.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( + when(pipelineApi.select(DependencyLabels.labelKey(DB, PATH))).thenReturn(Arrays.asList( pipeline("p1", "View", "owner1", IDENTIFIER), pipeline("p2", "View", "owner2", IDENTIFIER))); - when(triggerApi.select(PipelineDependencyLabels.labelKey(DB, PATH))).thenReturn( + when(triggerApi.select(DependencyLabels.labelKey(DB, PATH))).thenReturn( Collections.singletonList(trigger("t1", "LogicalTable", "lt-name", IDENTIFIER))); SQLException ex = assertThrows(SQLException.class, - () -> PipelineDependencyChecker.assertNoExternalDependents( + () -> DependencyChecker.assertNoExternalDependents( pipelineApi, triggerApi, DB, PATH, null, null)); assertTrue(ex.getMessage().contains("pipeline/p1")); assertTrue(ex.getMessage().contains("pipeline/p2")); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyLabelsTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyLabelsTest.java new file mode 100644 index 00000000..da9711c9 --- /dev/null +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyLabelsTest.java @@ -0,0 +1,178 @@ +package com.linkedin.hoptimator.k8s; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +import io.kubernetes.client.openapi.models.V1ObjectMeta; +import org.junit.jupiter.api.Test; + +import com.linkedin.hoptimator.Sink; +import com.linkedin.hoptimator.Source; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +class DependencyLabelsTest { + + private static Source src(String db, String... path) { + return new Source(db, Arrays.asList(path), Collections.emptyMap()); + } + + private static Sink sink(String db, String... path) { + return new Sink(db, Arrays.asList(path), Collections.emptyMap()); + } + + @Test + void identifierJoinsDatabaseAndPath() { + // Separator is "_" so the identifier is also a valid K8s label value out of the box. + assertEquals("mydb_a.b.c", DependencyLabels.identifier("mydb", Arrays.asList("a", "b", "c"))); + } + + @Test + void slugIsDeterministic() { + String s1 = DependencyLabels.slug("db", Arrays.asList("foo", "bar")); + String s2 = DependencyLabels.slug("db", Arrays.asList("foo", "bar")); + assertEquals(s1, s2); + } + + @Test + void slugVariesByDatabase() { + String a = DependencyLabels.slug("db1", Collections.singletonList("t")); + String b = DependencyLabels.slug("db2", Collections.singletonList("t")); + assertNotEquals(a, b); + } + + @Test + void slugVariesByPath() { + String a = DependencyLabels.slug("db", Arrays.asList("schema", "t")); + String b = DependencyLabels.slug("db", Arrays.asList("schema", "u")); + assertNotEquals(a, b); + } + + @Test + void labelKeyFitsKubernetesNameLimit() { + // Long path stressing the slug — name portion (after the /) must be ≤ 63 chars. + String key = DependencyLabels.labelKey( + "a-really-long-database-name", + Arrays.asList("catalog", "schema", "a_very_long_table_name_that_exceeds_sixty_three_chars")); + String namePortion = key.substring(key.indexOf('/') + 1); + assertTrue(namePortion.length() <= 63, "name portion must be ≤63 chars, got " + namePortion.length()); + assertTrue(namePortion.matches("[a-z0-9]([-a-z0-9_.]*[a-z0-9])?"), + "name portion must match K8s label-name regex, got: " + namePortion); + } + + private static V1ObjectMeta stamp(java.util.List sources, Sink sink) { + V1ObjectMeta meta = new V1ObjectMeta(); + DependencyLabels.stamp(meta, sources, sink); + return meta; + } + + @Test + void stampIncludesSourcesAndSink() { + // Both edges matter: dropping a source orphans readers; dropping a sink orphans writers. + V1ObjectMeta meta = stamp( + Arrays.asList(src("kafka1", "events"), src("venice1", "store")), + sink("mysql1", "outbox")); + + Map labels = meta.getLabels(); + assertEquals(3, labels.size()); + assertTrue(labels.containsKey(DependencyLabels.labelKey("kafka1", Collections.singletonList("events")))); + assertTrue(labels.containsKey(DependencyLabels.labelKey("venice1", Collections.singletonList("store")))); + assertTrue(labels.containsKey(DependencyLabels.labelKey("mysql1", Collections.singletonList("outbox")))); + } + + @Test + void stampHandlesNullSink() { + V1ObjectMeta meta = stamp(Collections.singletonList(src("db", "t")), null); + assertEquals(1, meta.getLabels().size()); + assertNull(meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINK)); + } + + @Test + void stampCollapsesSelfLoopIntoOneLabel() { + // Self-loop pipeline: source and sink share a slug, so the map collapses to one entry + // rather than producing duplicate keys. + V1ObjectMeta meta = stamp(Collections.singletonList(src("db", "t")), sink("db", "t")); + assertEquals(1, meta.getLabels().size()); + } + + @Test + void stampLabelValueTruncatedAtSixtyThreeChars() { + String longPath = "this_is_a_really_long_table_name_that_exceeds_sixty_three_chars_by_a_lot"; + V1ObjectMeta meta = stamp(Collections.singletonList(src("db", longPath)), null); + String value = meta.getLabels().values().iterator().next(); + assertTrue(value.length() <= 63); + } + + @Test + void stampLabelValueIsKubernetesLabelValueCompliant() { + // K8s label values must match (([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])? + // — the identifier separator is "_" precisely so this holds out of the box for typical + // (database, path) shapes seen in production. + V1ObjectMeta meta = stamp( + Collections.singletonList(src("ads-database", "ADS", "PAGE_VIEWS")), null); + String value = meta.getLabels().values().iterator().next(); + + assertTrue(value.length() <= 63); + assertTrue(value.matches("(([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])?"), + "value must satisfy K8s label-value regex, got: " + value); + assertFalse(value.contains("/"), "no '/' separator should leak into the label value"); + } + + @Test + void stampSourcesAnnotationListsOnlySources() { + V1ObjectMeta meta = stamp( + Arrays.asList(src("kafka", "a"), src("venice", "b")), sink("mysql", "c")); + String annotation = meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SOURCES); + assertTrue(annotation.contains("kafka_a")); + assertTrue(annotation.contains("venice_b")); + assertFalse(annotation.contains("mysql_c"), "sinks must not appear in sources annotation"); + } + + @Test + void stampSourcesAnnotationDeduplicatesIdenticalSources() { + V1ObjectMeta meta = stamp(Arrays.asList(src("db", "t"), src("db", "t")), null); + assertEquals("db_t", meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SOURCES)); + } + + @Test + void stampSinkAnnotationCarriesSinkIdentifier() { + V1ObjectMeta meta = stamp(Collections.emptyList(), sink("mysql", "c")); + assertEquals("mysql_c", meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINK)); + } + + @Test + void stampSkipsNullDatabaseSource() { + // Triggers may carry a source with a null database (e.g. table-name-only DROP); the stamp + // helper drops those rather than producing a "null_" identifier. + V1ObjectMeta meta = stamp( + Collections.singletonList(src(null, "t")), null); + assertTrue(meta.getLabels() == null || meta.getLabels().isEmpty(), + "no depends-on labels expected when source has null database"); + assertNull(meta.getAnnotations() == null ? null + : meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SOURCES)); + } + + @Test + void parseAnnotationRoundtripFromSourcesValue() { + V1ObjectMeta meta = stamp(Arrays.asList(src("a", "1"), src("b", "2")), null); + String annotation = meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SOURCES); + Set parsed = DependencyLabels.parseAnnotation(annotation); + assertEquals(2, parsed.size()); + assertTrue(parsed.contains("a_1")); + assertTrue(parsed.contains("b_2")); + } + + @Test + void parseAnnotationHandlesNullAndEmpty() { + assertTrue(DependencyLabels.parseAnnotation(null).isEmpty()); + assertTrue(DependencyLabels.parseAnnotation("").isEmpty()); + assertTrue(DependencyLabels.parseAnnotation(" , ").isEmpty()); + } +} diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidatorTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sDependencyValidatorTest.java similarity index 78% rename from hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidatorTest.java rename to hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sDependencyValidatorTest.java index ac94636d..f5e4ab92 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDependencyValidatorTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sDependencyValidatorTest.java @@ -24,20 +24,20 @@ /** - * Unit tests for {@link K8sPipelineDependencyValidator}. The validator is a thin adapter that - * forwards to {@link PipelineDependencyChecker#assertNoExternalDependents} — these tests use - * {@link MockedStatic} on both K8sContext and PipelineDependencyChecker to verify that source + * Unit tests for {@link K8sDependencyValidator}. The validator is a thin adapter that + * forwards to {@link DependencyChecker#assertNoExternalDependents} — these tests use + * {@link MockedStatic} on both K8sContext and DependencyChecker to verify that source * fields and self-owner fields are forwarded correctly, and that thrown SQLException becomes a * validation issue rather than propagating. */ @ExtendWith(MockitoExtension.class) -class K8sPipelineDependencyValidatorTest { +class K8sDependencyValidatorTest { @Mock private MockedStatic contextStatic; @Mock - private MockedStatic checkerStatic; + private MockedStatic checkerStatic; @Mock private Connection connection; @@ -53,8 +53,8 @@ private static Source source() { void validateForwardsSourceFieldsAndSelfOwnerToChecker() { contextStatic.when(() -> K8sContext.create(connection)).thenReturn(context); - K8sPipelineDependencyValidator validator = - new K8sPipelineDependencyValidator(source(), "LogicalTable", "my-table"); + K8sDependencyValidator validator = + new K8sDependencyValidator(source(), "LogicalTable", "my-table"); Validator.Issues issues = new Validator.Issues("test"); validator.validate(issues, connection); @@ -65,7 +65,7 @@ void validateForwardsSourceFieldsAndSelfOwnerToChecker() { ArgumentCaptor kindCaptor = ArgumentCaptor.forClass(String.class); ArgumentCaptor nameCaptor = ArgumentCaptor.forClass(String.class); - checkerStatic.verify(() -> PipelineDependencyChecker.assertNoExternalDependents( + checkerStatic.verify(() -> DependencyChecker.assertNoExternalDependents( eq(context), dbCaptor.capture(), pathCaptor.capture(), kindCaptor.capture(), nameCaptor.capture())); @@ -81,15 +81,15 @@ void validateForwardsSourceFieldsAndSelfOwnerToChecker() { void validatePassesNullSelfOwnerWhenUnset() { contextStatic.when(() -> K8sContext.create(connection)).thenReturn(context); - K8sPipelineDependencyValidator validator = - new K8sPipelineDependencyValidator(source(), null, null); + K8sDependencyValidator validator = + new K8sDependencyValidator(source(), null, null); Validator.Issues issues = new Validator.Issues("test"); validator.validate(issues, connection); ArgumentCaptor kindCaptor = ArgumentCaptor.forClass(String.class); ArgumentCaptor nameCaptor = ArgumentCaptor.forClass(String.class); - checkerStatic.verify(() -> PipelineDependencyChecker.assertNoExternalDependents( + checkerStatic.verify(() -> DependencyChecker.assertNoExternalDependents( eq(context), nullable(String.class), nullable(List.class), kindCaptor.capture(), nameCaptor.capture())); @@ -101,13 +101,13 @@ void validatePassesNullSelfOwnerWhenUnset() { @SuppressWarnings("unchecked") void validateRecordsCheckerSqlExceptionAsIssue() { contextStatic.when(() -> K8sContext.create(connection)).thenReturn(context); - checkerStatic.when(() -> PipelineDependencyChecker.assertNoExternalDependents( + checkerStatic.when(() -> DependencyChecker.assertNoExternalDependents( nullable(K8sContext.class), nullable(String.class), nullable(List.class), nullable(String.class), nullable(String.class))) .thenThrow(new SQLException("3 active pipeline(s) depend on it: p1, p2, p3")); - K8sPipelineDependencyValidator validator = - new K8sPipelineDependencyValidator(source(), null, null); + K8sDependencyValidator validator = + new K8sDependencyValidator(source(), null, null); Validator.Issues issues = new Validator.Issues("test"); validator.validate(issues, connection); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java index 08a9f1a2..7d9a5495 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java @@ -62,11 +62,11 @@ void stampsDependencyLabelsForSourcesAndSink() throws SQLException { assertEquals(3, labels.size(), "should have one label per source + one for the sink"); assertTrue(labels.containsKey( - PipelineDependencyLabels.labelKey("kafka1", Collections.singletonList("topic-a")))); + DependencyLabels.labelKey("kafka1", Collections.singletonList("topic-a")))); assertTrue(labels.containsKey( - PipelineDependencyLabels.labelKey("kafka2", Collections.singletonList("topic-b")))); + DependencyLabels.labelKey("kafka2", Collections.singletonList("topic-b")))); assertTrue(labels.containsKey( - PipelineDependencyLabels.labelKey("mysql", Collections.singletonList("outbox")))); + DependencyLabels.labelKey("mysql", Collections.singletonList("outbox")))); } @Test @@ -79,8 +79,8 @@ void stampsDirectionalAnnotations() throws SQLException { V1alpha1Pipeline pipeline = deployer.toK8sObject(); Map annotations = pipeline.getMetadata().getAnnotations(); - String sources = annotations.get(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES); - String sink = annotations.get(PipelineDependencyLabels.ANNOTATION_KEY_SINK); + String sources = annotations.get(DependencyLabels.ANNOTATION_KEY_SOURCES); + String sink = annotations.get(DependencyLabels.ANNOTATION_KEY_SINK); assertNotNull(sources); assertNotNull(sink); assertTrue(sources.contains("kafka_topic"), diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java index 31003784..7289aa18 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java @@ -100,7 +100,7 @@ void updateWithPausedOptionPausesTrigger() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -118,7 +118,7 @@ void updateWithPausedOptionUnpausesTrigger() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "false"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -136,7 +136,7 @@ void updateWithPausedOptionCreatesSpecIfNull() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -158,22 +158,22 @@ void updateWithPausedOptionAlsoStampsDependsOnLabelsWhenDatabaseSet() throws SQL Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), null, options, new Source("mysql-db", Arrays.asList("MYSQL", "testdb", "events"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), null, options, new Source("mysql-db", Arrays.asList("MYSQL", "testdb", "events"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); - String expectedLabel = PipelineDependencyLabels.labelKey( + String expectedLabel = DependencyLabels.labelKey( "mysql-db", Arrays.asList("MYSQL", "testdb", "events")); - String expectedIdentifier = PipelineDependencyLabels.identifier( + String expectedIdentifier = DependencyLabels.identifier( "mysql-db", Arrays.asList("MYSQL", "testdb", "events")); assertNotNull(existing.getMetadata().getLabels(), "labels must be set after partial update"); assertTrue(existing.getMetadata().getLabels().containsKey(expectedLabel), "depends-on label must be stamped on the partial-update path: " + existing.getMetadata().getLabels()); assertNotNull(existing.getMetadata().getAnnotations(), "annotations must be set"); assertEquals(expectedIdentifier, - existing.getMetadata().getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SOURCES)); + existing.getMetadata().getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SOURCES)); } @Test @@ -195,11 +195,11 @@ void updateStampsSinkLabelWhenTriggerCarriesASink() throws SQLException { K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); - String sourceLabel = PipelineDependencyLabels.labelKey( + String sourceLabel = DependencyLabels.labelKey( "hdfs-db", Arrays.asList("HDFS", "events")); - String sinkLabel = PipelineDependencyLabels.labelKey( + String sinkLabel = DependencyLabels.labelKey( "venice-db", Arrays.asList("VENICE", "events")); - String sinkIdentifier = PipelineDependencyLabels.identifier( + String sinkIdentifier = DependencyLabels.identifier( "venice-db", Arrays.asList("VENICE", "events")); assertTrue(existing.getMetadata().getLabels().containsKey(sourceLabel), @@ -207,7 +207,7 @@ void updateStampsSinkLabelWhenTriggerCarriesASink() throws SQLException { assertTrue(existing.getMetadata().getLabels().containsKey(sinkLabel), "sink-side depends-on label must be stamped: " + existing.getMetadata().getLabels()); assertEquals(sinkIdentifier, - existing.getMetadata().getAnnotations().get(PipelineDependencyLabels.ANNOTATION_KEY_SINK), + existing.getMetadata().getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINK), "depends-on-sink annotation must record the sink identifier verbatim"); } @@ -215,7 +215,7 @@ void updateStampsSinkLabelWhenTriggerCarriesASink() throws SQLException { void updateWithPausedOptionThrowsWhenTriggerNotFound() { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -229,7 +229,7 @@ void deleteRemovesTrigger() throws SQLException { .spec(new V1alpha1TableTriggerSpec()); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -240,7 +240,7 @@ void deleteRemovesTrigger() throws SQLException { @Test void deleteThrowsWhenTriggerNotFound() { - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -255,7 +255,7 @@ void toK8sObjectBuildsCorrectTrigger() throws SQLException { + "metadata:\n name: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -273,7 +273,7 @@ void toK8sObjectWithNoJobPropertiesHasNullJobProperties() throws SQLException { jobTemplates.add(jobTemplate); // No job.properties.* options — spec should NOT have jobProperties set - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -293,7 +293,7 @@ void toK8sObjectWithJobPropertiesIncludesThemInSpec() throws SQLException { Map options = new HashMap<>(); options.put("job.properties.parallelism", "4"); options.put("job.properties.restart-strategy", "never"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -316,7 +316,7 @@ void toK8sObjectOptionsPutAllIncludedInEnvironment() throws SQLException { Map options = new HashMap<>(); options.put("someKey", "someValue"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "5 4 * * *", options, new Source(null, Arrays.asList("SCHEMA", "MY_TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "5 4 * * *", options, new Source(null, Arrays.asList("SCHEMA", "MY_TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -339,7 +339,7 @@ void toK8sObjectForEachAppliesJobPropertiesFilter() throws SQLException { options.put("job.properties.key1", "val1"); options.put("job.properties.key2", "val2"); options.put("other.option", "ignored"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -361,7 +361,7 @@ void updateWithPausedOptionCallsApiUpdate() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -381,7 +381,7 @@ void updateWithChangedSpecCallsSuperUpdate() throws SQLException { jobTemplates.add(jobTemplate); // No PAUSED_OPTION → should fall through to super.update() - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); // super.update() calls api.update() via K8sDeployer; FakeK8sApi.update adds to list @@ -393,7 +393,7 @@ void updateWithChangedSpecCallsSuperUpdate() throws SQLException { @Test void deleteOnNonExistingTriggerThrowsSqlException() { // Graceful handling when trigger not found - Trigger trigger = new Trigger("NONEXISTENT", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("NONEXISTENT", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -413,7 +413,7 @@ void updatePreservesPausedWhenOptionsHaveNoPausedOption() throws SQLException { .spec(new V1alpha1TableTriggerSpec().paused(true)); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -430,7 +430,7 @@ void updatePreservesUnpausedWhenOptionsHaveNoPausedOption() throws SQLException .spec(new V1alpha1TableTriggerSpec().paused(false)); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -450,7 +450,7 @@ void updateWithPausedOptionFalseUnpausesAlreadyPausedTrigger() throws SQLExcepti Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "false"); - Trigger trigger = new Trigger("MY_TRIGGER", null, null, options, new Source(null, new ArrayList<>(), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", null, null, options, new Source(null, new ArrayList<>(), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -469,7 +469,7 @@ void updateFallsThroughToSuperUpdateWhenNoExistingAndNoPausedOption() throws SQL .spec(new V1alpha1JobTemplateSpec().yaml("template: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -493,7 +493,7 @@ void updateFallsThroughToSuperUpdateWhenExistingHasNullPaused() throws SQLExcept .spec(new V1alpha1JobTemplateSpec().yaml("template: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -515,7 +515,7 @@ void toK8sObjectSetsSpecPausedWhenPausedOptionTrue() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap())); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sValidatorProviderTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sValidatorProviderTest.java index a4cf0189..52372e6b 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sValidatorProviderTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sValidatorProviderTest.java @@ -29,7 +29,7 @@ void returnsDependencyValidatorForPendingDeleteOfSource() { Collection validators = provider.validators(pd, null); assertEquals(1, validators.size()); - assertInstanceOf(K8sPipelineDependencyValidator.class, validators.iterator().next()); + assertInstanceOf(K8sDependencyValidator.class, validators.iterator().next()); } @Test @@ -42,7 +42,7 @@ void returnsDependencyValidatorWhenSelfOwnerIsSet() { Collection validators = provider.validators(pd, null); assertEquals(1, validators.size()); - assertInstanceOf(K8sPipelineDependencyValidator.class, validators.iterator().next()); + assertInstanceOf(K8sDependencyValidator.class, validators.iterator().next()); } @Test diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java deleted file mode 100644 index ce126558..00000000 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/PipelineDependencyLabelsTest.java +++ /dev/null @@ -1,163 +0,0 @@ -package com.linkedin.hoptimator.k8s; - -import java.util.Arrays; -import java.util.Collections; -import java.util.Map; -import java.util.Set; - -import org.junit.jupiter.api.Test; - -import com.linkedin.hoptimator.Sink; -import com.linkedin.hoptimator.Source; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; - - -class PipelineDependencyLabelsTest { - - private static Source src(String db, String... path) { - return new Source(db, Arrays.asList(path), Collections.emptyMap()); - } - - private static Sink sink(String db, String... path) { - return new Sink(db, Arrays.asList(path), Collections.emptyMap()); - } - - @Test - void identifierJoinsDatabaseAndPath() { - // Separator is "_" so the identifier is also a valid K8s label value out of the box. - assertEquals("mydb_a.b.c", PipelineDependencyLabels.identifier("mydb", Arrays.asList("a", "b", "c"))); - } - - @Test - void slugIsDeterministic() { - String s1 = PipelineDependencyLabels.slug("db", Arrays.asList("foo", "bar")); - String s2 = PipelineDependencyLabels.slug("db", Arrays.asList("foo", "bar")); - assertEquals(s1, s2); - } - - @Test - void slugVariesByDatabase() { - String a = PipelineDependencyLabels.slug("db1", Collections.singletonList("t")); - String b = PipelineDependencyLabels.slug("db2", Collections.singletonList("t")); - assertNotEquals(a, b); - } - - @Test - void slugVariesByPath() { - String a = PipelineDependencyLabels.slug("db", Arrays.asList("schema", "t")); - String b = PipelineDependencyLabels.slug("db", Arrays.asList("schema", "u")); - assertNotEquals(a, b); - } - - @Test - void labelKeyFitsKubernetesNameLimit() { - // Long path stressing the slug — name portion (after the /) must be ≤ 63 chars. - String key = PipelineDependencyLabels.labelKey( - "a-really-long-database-name", - Arrays.asList("catalog", "schema", "a_very_long_table_name_that_exceeds_sixty_three_chars")); - String namePortion = key.substring(key.indexOf('/') + 1); - assertTrue(namePortion.length() <= 63, "name portion must be ≤63 chars, got " + namePortion.length()); - assertTrue(namePortion.matches("[a-z0-9]([-a-z0-9_.]*[a-z0-9])?"), - "name portion must match K8s label-name regex, got: " + namePortion); - } - - @Test - void labelsForIncludesSourcesAndSink() { - // Both edges matter: dropping a source orphans readers; dropping a sink orphans writers. - Source s1 = src("kafka1", "events"); - Source s2 = src("venice1", "store"); - Sink sink = sink("mysql1", "outbox"); - Map labels = PipelineDependencyLabels.labelsFor(Arrays.asList(s1, s2), sink); - - assertEquals(3, labels.size()); - assertTrue(labels.containsKey(PipelineDependencyLabels.labelKey("kafka1", Collections.singletonList("events")))); - assertTrue(labels.containsKey(PipelineDependencyLabels.labelKey("venice1", Collections.singletonList("store")))); - assertTrue(labels.containsKey(PipelineDependencyLabels.labelKey("mysql1", Collections.singletonList("outbox")))); - } - - @Test - void labelsForHandlesNullSink() { - Map labels = PipelineDependencyLabels.labelsFor( - Collections.singletonList(src("db", "t")), null); - assertEquals(1, labels.size()); - } - - @Test - void labelsForCollapsesSelfLoopIntoOneEntry() { - // Self-loop pipeline: source and sink share a slug, so the map collapses to one entry - // rather than producing duplicate keys. - Source s = src("db", "t"); - Sink k = sink("db", "t"); - Map labels = PipelineDependencyLabels.labelsFor(Collections.singletonList(s), k); - assertEquals(1, labels.size()); - } - - @Test - void labelValueTruncatedAtSixtyThreeChars() { - String longPath = "this_is_a_really_long_table_name_that_exceeds_sixty_three_chars_by_a_lot"; - Map labels = PipelineDependencyLabels.labelsFor( - Collections.singletonList(src("db", longPath)), null); - String value = labels.values().iterator().next(); - assertTrue(value.length() <= 63); - } - - @Test - void labelValueIsKubernetesLabelValueCompliant() { - // K8s label values must match (([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])? - // — the identifier separator is "_" precisely so this holds out of the box for typical - // (database, path) shapes seen in production. - Map labels = PipelineDependencyLabels.labelsFor( - Collections.singletonList(src("ads-database", "ADS", "PAGE_VIEWS")), null); - String value = labels.values().iterator().next(); - - assertTrue(value.length() <= 63); - assertTrue(value.matches("(([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])?"), - "value must satisfy K8s label-value regex, got: " + value); - assertFalse(value.contains("/"), "no '/' separator should leak into the label value"); - } - - @Test - void sourcesAnnotationListsOnlySources() { - String annotation = PipelineDependencyLabels.sourcesAnnotation( - Arrays.asList(src("kafka", "a"), src("venice", "b"))); - assertTrue(annotation.contains("kafka_a")); - assertTrue(annotation.contains("venice_b")); - assertFalse(annotation.contains("mysql_c"), "sinks must not appear in sources annotation"); - } - - @Test - void sourcesAnnotationDeduplicatesIdenticalSources() { - String annotation = PipelineDependencyLabels.sourcesAnnotation( - Arrays.asList(src("db", "t"), src("db", "t"))); - assertEquals("db_t", annotation); - } - - @Test - void sinkAnnotationReturnsSingleIdentifierOrNull() { - assertEquals("mysql_c", PipelineDependencyLabels.sinkAnnotation(sink("mysql", "c"))); - assertNull(PipelineDependencyLabels.sinkAnnotation(null), - "sink annotation should be null when there's no sink"); - } - - @Test - void parseAnnotationRoundtripFromSourcesValue() { - String annotation = PipelineDependencyLabels.sourcesAnnotation( - Arrays.asList(src("a", "1"), src("b", "2"))); - Set parsed = PipelineDependencyLabels.parseAnnotation(annotation); - assertEquals(2, parsed.size()); - assertTrue(parsed.contains("a_1")); - assertTrue(parsed.contains("b_2")); - } - - @Test - void parseAnnotationHandlesNullAndEmpty() { - assertTrue(PipelineDependencyLabels.parseAnnotation(null).isEmpty()); - assertTrue(PipelineDependencyLabels.parseAnnotation("").isEmpty()); - assertTrue(PipelineDependencyLabels.parseAnnotation(" , ").isEmpty()); - } -} From 5fbbe4168e3876e0e45a1620371357a5dedcfa47 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Tue, 12 May 2026 12:36:18 -0400 Subject: [PATCH 13/15] refactoring --- .../hoptimator/jdbc/HoptimatorDdlExecutor.java | 3 --- .../hoptimator/k8s/K8sTriggerDeployer.java | 18 +++--------------- 2 files changed, 3 insertions(+), 18 deletions(-) diff --git a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java index 8ce3f938..47b3f871 100644 --- a/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java +++ b/hoptimator-jdbc/src/main/java/com/linkedin/hoptimator/jdbc/HoptimatorDdlExecutor.java @@ -229,9 +229,6 @@ public void execute(SqlCreateTrigger create, CalcitePrepare.Context context) { String cronSchedule = create.cron != null ? ((SqlLiteral) create.cron).getValueAs(String.class) : null; UserJob job = new UserJob(jobNamespace, jobName); - // Resolve the underlying database name the target table belongs to so the deployer can - // stamp depends-on labels: same identifier shape as Pipelines, so the dep-guard finds the - // trigger when somebody tries to drop the source it points at. Source source = new Source(databaseOf(target), targetPath, Collections.emptyMap()); Trigger trigger = new Trigger(name, job, cronSchedule, options, source, null); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java index 2f9bf44f..c9666cc8 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployer.java @@ -19,9 +19,6 @@ public class K8sTriggerDeployer extends K8sDeployer { - /** Annotation carrying the JobTemplate name (e.g. {@code retl-job-template}) used to render this trigger. */ - static final String JOB_TEMPLATE_ANNOTATION = "hoptimator.linkedin.com/job-template"; - private final K8sContext context; private final Trigger trigger; private final K8sApi triggerApi; @@ -67,7 +64,7 @@ public void update() throws SQLException { spec.setPaused(targetPaused); // Refresh dependency-tracking labels and annotation here too — without this, the partial // update path (used when the LogicalTable is re-applied) would leave triggers with stale - // or missing depends-on metadata, breaking the visualizer's reverse lookup. + // or missing depends-on metadata. stampDependencyMetadata(existingTrigger); triggerApi.update(existingTrigger); return; @@ -85,11 +82,6 @@ private void stampDependencyMetadata(V1alpha1TableTrigger target) { DependencyLabels.stamp(meta, source != null ? Collections.singletonList(source) : Collections.emptyList(), trigger.sink()); - if (trigger.job() != null) { - Map annotations = meta.getAnnotations() != null ? meta.getAnnotations() : new HashMap<>(); - annotations.put(JOB_TEMPLATE_ANNOTATION, trigger.job().name()); - meta.setAnnotations(annotations); - } } @Override @@ -126,16 +118,12 @@ protected V1alpha1TableTrigger toK8sObject() throws SQLException { Map labels = new HashMap<>(); labels.put("view", viewName); // a corresponding view object may or may not exist. meta.setLabels(labels); - // Stamp depends-on labels so the visualizer (and the dep-guard) can find triggers via + // Stamp depends-on labels so the dep-guard can find triggers via // label selector. The Trigger's source is the upstream table the job reads from. When the - // trigger carries a Sink (set by LogicalTableDeployer for bridging triggers), we - // additionally stamp the sink — that's what makes reverse-ETL flows render as connectors. + // trigger carries a Sink, we additionally stamp the sink. DependencyLabels.stamp(meta, source != null ? Collections.singletonList(source) : Collections.emptyList(), trigger.sink()); - Map annotations = meta.getAnnotations() != null ? meta.getAnnotations() : new HashMap<>(); - annotations.put(JOB_TEMPLATE_ANNOTATION, trigger.job().name()); - meta.setAnnotations(annotations); String template = jobTemplate.getSpec().getYaml(); String rendered = new Template.SimpleTemplate(template).render(env); Map jobProps = new HashMap<>(); From 64a37942732c1f81261bcedb17a8333720e57fc7 Mon Sep 17 00:00:00 2001 From: Joseph Grogan Date: Tue, 12 May 2026 13:02:42 -0400 Subject: [PATCH 14/15] Allow multiple sources/sinks (future proofing) --- .../hoptimator/k8s/DependencyChecker.java | 8 +-- .../hoptimator/k8s/DependencyLabels.java | 54 +++++++++++-------- .../hoptimator/k8s/K8sPipelineDeployer.java | 3 +- .../hoptimator/k8s/K8sTriggerDeployer.java | 4 +- .../hoptimator/k8s/DependencyLabelsTest.java | 7 +-- .../k8s/K8sPipelineDeployerTest.java | 2 +- .../k8s/K8sTriggerDeployerTest.java | 4 +- .../test/resources/kafka-ddl-create-table.id | 4 +- .../logical/LogicalTableDeployer.java | 2 +- .../src/test/resources/logical-ddl.id | 4 +- 10 files changed, 51 insertions(+), 41 deletions(-) diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyChecker.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyChecker.java index cd2ecf8c..b539b8e2 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyChecker.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyChecker.java @@ -22,7 +22,7 @@ * {@link com.linkedin.hoptimator.Deployer} is about to delete. * *

      Both CRDs carry the same {@code depends-on-} label and {@code depends-on-sources}/ - * {@code depends-on-sink} annotations (stamped by {@link K8sPipelineDeployer} and + * {@code depends-on-sinks} annotations (stamped by {@link K8sPipelineDeployer} and * {@link K8sTriggerDeployer}), so the same lookup works for either: a label-selector list against * the CRD group is O(matches) on the wire, then each candidate is cross-checked against the union * of the source + sink annotations to rule out hash collisions in the slug and stale labels left @@ -115,14 +115,14 @@ private static boolean annotationConfirms(V1ObjectMeta meta, String identifier) return true; // pre-labeling resource — conservatively trust the label match } String sourcesAnno = meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SOURCES); - String sinkAnno = meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINK); - if (sourcesAnno == null && sinkAnno == null) { + String sinksAnno = meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINKS); + if (sourcesAnno == null && sinksAnno == null) { return true; // same — no annotations to cross-check against } if (sourcesAnno != null && DependencyLabels.parseAnnotation(sourcesAnno).contains(identifier)) { return true; } - return identifier.equals(sinkAnno); + return sinksAnno != null && DependencyLabels.parseAnnotation(sinksAnno).contains(identifier); } /** diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyLabels.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyLabels.java index 74ee3f77..93d3ff70 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyLabels.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/DependencyLabels.java @@ -4,13 +4,12 @@ import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; -import javax.annotation.Nullable; - import io.kubernetes.client.openapi.models.V1ObjectMeta; import com.linkedin.hoptimator.Sink; @@ -19,8 +18,8 @@ /** * Encodes a resource's dependency edges as K8s labels + annotations. Used by both - * {@link K8sPipelineDeployer} and {@link K8sTriggerDeployer} so the dep-guard and - * the visualizer can find dependents with a single label-selector query. + * {@link K8sPipelineDeployer} and {@link K8sTriggerDeployer} so the dep-guard can + * find dependents with a single label-selector query. * *

      Every source and sink the resource references becomes a label: * {@code hoptimator.linkedin.com/depends-on-: "_"} where @@ -31,7 +30,7 @@ *

      Two annotations preserve the directional information the labels lose: *

        *
      • {@link #ANNOTATION_KEY_SOURCES} — comma-separated list of source identifiers verbatim.
      • - *
      • {@link #ANNOTATION_KEY_SINK} — the single sink identifier verbatim.
      • + *
      • {@link #ANNOTATION_KEY_SINKS} — comma-separated list of sink identifiers verbatim.
      • *
      * Together they serve two purposes: *
        @@ -45,10 +44,10 @@ public final class DependencyLabels { static final String LABEL_PREFIX = "hoptimator.linkedin.com/depends-on-"; - /** Annotation listing only source identifiers. */ + /** Annotation listing source identifiers, comma-separated. */ public static final String ANNOTATION_KEY_SOURCES = "hoptimator.linkedin.com/depends-on-sources"; - /** Annotation holding the single sink identifier. */ - public static final String ANNOTATION_KEY_SINK = "hoptimator.linkedin.com/depends-on-sink"; + /** Annotation listing sink identifiers, comma-separated. */ + public static final String ANNOTATION_KEY_SINKS = "hoptimator.linkedin.com/depends-on-sinks"; private static final int SLUG_LENGTH = 16; // 64 bits of SHA-256 → ~1 in 1.8e19 collisions private static final int MAX_LABEL_VALUE = 63; @@ -83,32 +82,41 @@ public static String labelKey(String database, Iterable path) { * existing labels/annotations on the object. Both edges matter to the guard: dropping a source * orphans resources that read from it; dropping a sink orphans resources that write to it. * - *

        Sources whose {@code database()} is null are skipped — they don't have a stable identifier. + *

        Sources and sinks whose {@code database()} is null are skipped — they don't have a stable + * identifier. Callers with a single sink should pass {@link Collections#singletonList}. */ - public static void stamp(V1ObjectMeta meta, Collection sources, @Nullable Sink sink) { + public static void stamp(V1ObjectMeta meta, Collection sources, Collection sinks) { Map labels = meta.getLabels() != null ? meta.getLabels() : new HashMap<>(); Map annotations = meta.getAnnotations() != null ? meta.getAnnotations() : new HashMap<>(); - Set sourceIds = new LinkedHashSet<>(); - for (Source src : sources) { - if (src == null || src.database() == null) { - continue; - } - String id = identifier(src.database(), src.path()); - labels.put(labelKey(src.database(), src.path()), truncate(id)); - sourceIds.add(id); - } + Set sourceIds = collectIdentifiers(sources, labels); + Set sinkIds = collectIdentifiers(sinks, labels); if (!sourceIds.isEmpty()) { annotations.put(ANNOTATION_KEY_SOURCES, String.join(",", sourceIds)); } - if (sink != null && sink.database() != null) { - String id = identifier(sink.database(), sink.path()); - labels.put(labelKey(sink.database(), sink.path()), truncate(id)); - annotations.put(ANNOTATION_KEY_SINK, id); + if (!sinkIds.isEmpty()) { + annotations.put(ANNOTATION_KEY_SINKS, String.join(",", sinkIds)); } meta.setLabels(labels); meta.setAnnotations(annotations); } + private static Set collectIdentifiers(Collection deps, + Map labels) { + Set ids = new LinkedHashSet<>(); + if (deps == null) { + return ids; + } + for (Source dep : deps) { + if (dep == null || dep.database() == null) { + continue; + } + String id = identifier(dep.database(), dep.path()); + labels.put(labelKey(dep.database(), dep.path()), truncate(id)); + ids.add(id); + } + return ids; + } + /** Parses the collision-guard annotation back into the set of identifiers it encoded. */ public static Set parseAnnotation(String annotation) { Set out = new LinkedHashSet<>(); diff --git a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java index f9f7e840..4f2be404 100644 --- a/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java +++ b/hoptimator-k8s/src/main/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployer.java @@ -46,7 +46,8 @@ class K8sPipelineDeployer extends K8sDeployer jobProps = new HashMap<>(); diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyLabelsTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyLabelsTest.java index da9711c9..eea31ef3 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyLabelsTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/DependencyLabelsTest.java @@ -69,7 +69,8 @@ void labelKeyFitsKubernetesNameLimit() { private static V1ObjectMeta stamp(java.util.List sources, Sink sink) { V1ObjectMeta meta = new V1ObjectMeta(); - DependencyLabels.stamp(meta, sources, sink); + DependencyLabels.stamp(meta, sources, + sink == null ? Collections.emptyList() : Collections.singletonList(sink)); return meta; } @@ -91,7 +92,7 @@ void stampIncludesSourcesAndSink() { void stampHandlesNullSink() { V1ObjectMeta meta = stamp(Collections.singletonList(src("db", "t")), null); assertEquals(1, meta.getLabels().size()); - assertNull(meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINK)); + assertNull(meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINKS)); } @Test @@ -144,7 +145,7 @@ void stampSourcesAnnotationDeduplicatesIdenticalSources() { @Test void stampSinkAnnotationCarriesSinkIdentifier() { V1ObjectMeta meta = stamp(Collections.emptyList(), sink("mysql", "c")); - assertEquals("mysql_c", meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINK)); + assertEquals("mysql_c", meta.getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINKS)); } @Test diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java index 7d9a5495..1eebc401 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sPipelineDeployerTest.java @@ -80,7 +80,7 @@ void stampsDirectionalAnnotations() throws SQLException { Map annotations = pipeline.getMetadata().getAnnotations(); String sources = annotations.get(DependencyLabels.ANNOTATION_KEY_SOURCES); - String sink = annotations.get(DependencyLabels.ANNOTATION_KEY_SINK); + String sink = annotations.get(DependencyLabels.ANNOTATION_KEY_SINKS); assertNotNull(sources); assertNotNull(sink); assertTrue(sources.contains("kafka_topic"), diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java index 7289aa18..0bdb598d 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java @@ -150,7 +150,7 @@ void updateWithPausedOptionCreatesSpecIfNull() throws SQLException { void updateWithPausedOptionAlsoStampsDependsOnLabelsWhenDatabaseSet() throws SQLException { // The partial-update path (paused-only) used to skip toK8sObject and never refresh the // depends-on metadata. Pin down that re-applying the LogicalTable through this path now - // stamps the labels/annotation so visualization (and the dep-guard reverse lookup) works. + // stamps the labels/annotation V1alpha1TableTrigger existing = new V1alpha1TableTrigger() .metadata(new V1ObjectMeta().name("mytrigger")) .spec(new V1alpha1TableTriggerSpec().paused(true)); @@ -207,7 +207,7 @@ void updateStampsSinkLabelWhenTriggerCarriesASink() throws SQLException { assertTrue(existing.getMetadata().getLabels().containsKey(sinkLabel), "sink-side depends-on label must be stamped: " + existing.getMetadata().getLabels()); assertEquals(sinkIdentifier, - existing.getMetadata().getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINK), + existing.getMetadata().getAnnotations().get(DependencyLabels.ANNOTATION_KEY_SINKS), "depends-on-sink annotation must record the sink identifier verbatim"); } diff --git a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id index 38304532..e57ec7f7 100644 --- a/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id +++ b/hoptimator-kafka/src/test/resources/kafka-ddl-create-table.id @@ -104,12 +104,12 @@ create or replace materialized view KAFKA."create-table-test$guard" as select * # Source-side guard: existing-topic-2 is a source of the MV's pipeline. drop table KAFKA."existing-topic-2"; -active pipeline(s) depend on it +active dependent !error # Sink-side guard: create-table-test is the partial-view sink of the MV's pipeline. drop table KAFKA."create-table-test"; -active pipeline(s) depend on it +active dependent !error # Drop the dependent MV first; its pipeline (and both labels) go away. diff --git a/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java b/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java index 54cfff37..6cfc0004 100644 --- a/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java +++ b/hoptimator-logical/src/main/java/com/linkedin/hoptimator/logical/LogicalTableDeployer.java @@ -540,7 +540,7 @@ private void deployImplicitTrigger(Map tierMap, Map Date: Tue, 12 May 2026 13:30:02 -0400 Subject: [PATCH 15/15] Fix checkstyle --- .../k8s/K8sTriggerDeployerTest.java | 60 ++++++++++++------- 1 file changed, 40 insertions(+), 20 deletions(-) diff --git a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java index 0bdb598d..b2c60ab7 100644 --- a/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java +++ b/hoptimator-k8s/src/test/java/com/linkedin/hoptimator/k8s/K8sTriggerDeployerTest.java @@ -100,7 +100,8 @@ void updateWithPausedOptionPausesTrigger() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -118,7 +119,8 @@ void updateWithPausedOptionUnpausesTrigger() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "false"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -136,7 +138,8 @@ void updateWithPausedOptionCreatesSpecIfNull() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -158,7 +161,8 @@ void updateWithPausedOptionAlsoStampsDependsOnLabelsWhenDatabaseSet() throws SQL Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), null, options, new Source("mysql-db", Arrays.asList("MYSQL", "testdb", "events"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), null, options, + new Source("mysql-db", Arrays.asList("MYSQL", "testdb", "events"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -215,7 +219,8 @@ void updateStampsSinkLabelWhenTriggerCarriesASink() throws SQLException { void updateWithPausedOptionThrowsWhenTriggerNotFound() { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -229,7 +234,8 @@ void deleteRemovesTrigger() throws SQLException { .spec(new V1alpha1TableTriggerSpec()); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -240,7 +246,8 @@ void deleteRemovesTrigger() throws SQLException { @Test void deleteThrowsWhenTriggerNotFound() { - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -255,7 +262,8 @@ void toK8sObjectBuildsCorrectTrigger() throws SQLException { + "metadata:\n name: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -273,7 +281,8 @@ void toK8sObjectWithNoJobPropertiesHasNullJobProperties() throws SQLException { jobTemplates.add(jobTemplate); // No job.properties.* options — spec should NOT have jobProperties set - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -293,7 +302,8 @@ void toK8sObjectWithJobPropertiesIncludesThemInSpec() throws SQLException { Map options = new HashMap<>(); options.put("job.properties.parallelism", "4"); options.put("job.properties.restart-strategy", "never"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, + new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -316,7 +326,8 @@ void toK8sObjectOptionsPutAllIncludedInEnvironment() throws SQLException { Map options = new HashMap<>(); options.put("someKey", "someValue"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "5 4 * * *", options, new Source(null, Arrays.asList("SCHEMA", "MY_TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "5 4 * * *", options, + new Source(null, Arrays.asList("SCHEMA", "MY_TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -339,7 +350,8 @@ void toK8sObjectForEachAppliesJobPropertiesFilter() throws SQLException { options.put("job.properties.key1", "val1"); options.put("job.properties.key2", "val2"); options.put("other.option", "ignored"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, + new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify(); @@ -361,7 +373,8 @@ void updateWithPausedOptionCallsApiUpdate() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob(null, "myjob"), "0 * * * *", options, + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -381,7 +394,8 @@ void updateWithChangedSpecCallsSuperUpdate() throws SQLException { jobTemplates.add(jobTemplate); // No PAUSED_OPTION → should fall through to super.update() - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); // super.update() calls api.update() via K8sDeployer; FakeK8sApi.update adds to list @@ -393,7 +407,8 @@ void updateWithChangedSpecCallsSuperUpdate() throws SQLException { @Test void deleteOnNonExistingTriggerThrowsSqlException() { // Graceful handling when trigger not found - Trigger trigger = new Trigger("NONEXISTENT", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("NONEXISTENT", new UserJob(null, "myjob"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); @@ -413,7 +428,8 @@ void updatePreservesPausedWhenOptionsHaveNoPausedOption() throws SQLException { .spec(new V1alpha1TableTriggerSpec().paused(true)); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -430,7 +446,8 @@ void updatePreservesUnpausedWhenOptionsHaveNoPausedOption() throws SQLException .spec(new V1alpha1TableTriggerSpec().paused(false)); triggers.add(existing); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "myjob"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("schema", "table"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -469,7 +486,8 @@ void updateFallsThroughToSuperUpdateWhenNoExistingAndNoPausedOption() throws SQL .spec(new V1alpha1JobTemplateSpec().yaml("template: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -493,7 +511,8 @@ void updateFallsThroughToSuperUpdateWhenExistingHasNullPaused() throws SQLExcept .spec(new V1alpha1JobTemplateSpec().yaml("template: {{name}}")); jobTemplates.add(jobTemplate); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", Collections.emptyMap(), + new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); deployer.update(); @@ -515,7 +534,8 @@ void toK8sObjectSetsSpecPausedWhenPausedOptionTrue() throws SQLException { Map options = new HashMap<>(); options.put(Trigger.PAUSED_OPTION, "true"); - Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); + Trigger trigger = new Trigger("MY_TRIGGER", new UserJob("test-ns", "MY_JOB"), "0 * * * *", options, + new Source(null, Arrays.asList("SCHEMA", "TABLE"), Collections.emptyMap()), null); K8sTriggerDeployer deployer = makeDeployer(trigger, mockContext); List specs = deployer.specify();