From 43056f8f6b995ed77d1cb2905232b082dbf62845 Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Fri, 18 Oct 2024 04:28:07 -0700 Subject: [PATCH 01/11] Define Gobblin-on-Temporal`WorkforcePlan` and dynamic `ScalingDirective`s with parser --- .../GobblinTemporalClusterManager.java | 2 +- .../temporal/dynscale/ProfileDerivation.java | 54 ++++ .../temporal/dynscale/ProfileOverlay.java | 150 +++++++++++ .../temporal/dynscale/ScalingDirective.java | 44 ++++ .../dynscale/ScalingDirectiveParser.java | 132 ++++++++++ .../temporal/dynscale/StaffingDeltas.java | 39 +++ .../temporal/dynscale/WorkerProfile.java | 28 ++ .../temporal/dynscale/WorkforcePlan.java | 148 +++++++++++ .../temporal/dynscale/WorkforceProfiles.java | 76 ++++++ .../temporal/dynscale/WorkforceStaffing.java | 99 +++++++ .../dynscale/ProfileDerivationTest.java | 78 ++++++ .../temporal/dynscale/ProfileOverlayTest.java | 100 +++++++ .../dynscale/ScalingDirectiveParserTest.java | 247 ++++++++++++++++++ .../temporal/dynscale/WorkforcePlanTest.java | 187 +++++++++++++ .../dynscale/WorkforceStaffingTest.java | 95 +++++++ 15 files changed, 1478 insertions(+), 1 deletion(-) create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffingTest.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalClusterManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalClusterManager.java index a460bb42026..19a65078909 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalClusterManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalClusterManager.java @@ -224,7 +224,7 @@ public synchronized void stop() { this.stopStatus.setStopInprogress(true); - log.info("Stopping the Gobblin Cluster Manager"); + log.info("Stopping the Gobblin Temporal Cluster Manager"); stopAppLauncherAndServices(); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java new file mode 100644 index 00000000000..0e192c2c784 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.Optional; +import java.util.function.Function; + +import com.typesafe.config.Config; +import lombok.Data; +import lombok.Getter; + + +@Data +public class ProfileDerivation { + public static class UnknownBasisException extends Exception { + @Getter + private final String name; + public UnknownBasisException(String basisName) { + super("named '" + WorkforceProfiles.renderName(basisName) + "'"); + this.name = basisName; + } + } + + private final String basisProfileName; + private final ProfileOverlay overlay; + + public Config formulateConfig(Function> basisResolver) throws UnknownBasisException { + Optional optProfile = basisResolver.apply(basisProfileName); + if (!optProfile.isPresent()) { + throw new UnknownBasisException(basisProfileName); + } else { + return overlay.applyOverlay(optProfile.get().getConfig()); + } + } + + public String renderName() { + return WorkforceProfiles.renderName(this.basisProfileName); + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java new file mode 100644 index 00000000000..ed36ee9c16f --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; +import lombok.Data; + + +public interface ProfileOverlay { + + Config applyOverlay(Config config); + + ProfileOverlay over(ProfileOverlay other); + + + @Data + class KVPair { + private final String key; + private final String value; + } + + + @Data + class Adding implements ProfileOverlay { + private final List additionPairs; + + @Override + public Config applyOverlay(Config config) { + return additionPairs.stream().sequential().reduce(config, + (currConfig, additionPair) -> + currConfig.withValue(additionPair.getKey(), ConfigValueFactory.fromAnyRef(additionPair.getValue())), + (configA, configB) -> + configB.withFallback(configA) + ); + } + + @Override + public ProfileOverlay over(ProfileOverlay other) { + if (other instanceof Adding) { + Map base = ((Adding) other).getAdditionPairs().stream().collect(Collectors.toMap(KVPair::getKey, KVPair::getValue)); + additionPairs.stream().forEach(additionPair -> + base.put(additionPair.getKey(), additionPair.getValue())); + return new Adding(base.entrySet().stream().map(entry -> new KVPair(entry.getKey(), entry.getValue())).collect(Collectors.toList())); + } else if (other instanceof Removing) { + return Combo.normalize(this, (Removing) other); + } else if (other instanceof Combo) { + Combo otherCombo = (Combo) other; + return Combo.normalize((Adding) this.over(otherCombo.getAdding()), otherCombo.getRemoving()); + } else { + throw new IllegalArgumentException("unknown derived class of type '" + other.getClass().getName() + "': " + other); + } + } + } + + + @Data + class Removing implements ProfileOverlay { + private final List removalKeys; + + @Override + public Config applyOverlay(Config config) { + return removalKeys.stream().sequential().reduce(config, + (currConfig, removalKey) -> + currConfig.withoutPath(removalKey), + (configA, configB) -> + configA.withFallback(configB) + ); + } + + @Override + public ProfileOverlay over(ProfileOverlay other) { + if (other instanceof Adding) { + return Combo.normalize((Adding) other, this); + } else if (other instanceof Removing) { + Set otherKeys = new HashSet(((Removing) other).getRemovalKeys()); + otherKeys.addAll(removalKeys); + return new Removing(new ArrayList<>(otherKeys)); + } else if (other instanceof Combo) { + Combo otherCombo = (Combo) other; + return Combo.normalize(otherCombo.getAdding(), (Removing) this.over(otherCombo.getRemoving())); + } else { + throw new IllegalArgumentException("unknown derived class of type '" + other.getClass().getName() + "': " + other); + } + } + } + + + @Data + class Combo implements ProfileOverlay { + private final Adding adding; + private final Removing removing; + + // merely restrict access modifier from `public` to `protected`, as not meant to be instantiated outside this enclosing interface + private Combo(Adding adding, Removing removing) { + this.adding = adding; + this.removing = removing; + } + + protected static Combo normalize(Adding toAdd, Removing toRemove) { + // pre-remove any in `toAdd` that are also in `toRemove`... yet still maintain them in `toRemove`, in case the eventual `Config` "basis" also has any + Set removeKeysLookup = toRemove.getRemovalKeys().stream().collect(Collectors.toSet()); + List unmatchedAdditionPairs = toAdd.getAdditionPairs().stream().sequential().filter(additionPair -> + !removeKeysLookup.contains(additionPair.getKey()) + ).collect(Collectors.toList()); + return new Combo(new Adding(unmatchedAdditionPairs), new Removing(new ArrayList<>(removeKeysLookup))); + } + + @Override + public Config applyOverlay(Config config) { + return adding.applyOverlay(removing.applyOverlay(config)); + } + + @Override + public ProfileOverlay over(ProfileOverlay other) { + if (other instanceof Adding) { + return Combo.normalize((Adding) this.adding.over((Adding) other), this.removing); + } else if (other instanceof Removing) { + return Combo.normalize(this.adding, (Removing) this.removing.over((Removing) other)); + } else if (other instanceof Combo) { + Combo otherCombo = (Combo) other; + return Combo.normalize((Adding) this.adding.over(otherCombo.getAdding()), (Removing) this.removing.over(otherCombo.getRemoving())); + } else { + throw new IllegalArgumentException("unknown derived class of type '" + other.getClass().getName() + "': " + other); + } + } + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java new file mode 100644 index 00000000000..77d9b483c9a --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.Optional; +import lombok.Data; +import lombok.RequiredArgsConstructor; + + +@Data +@RequiredArgsConstructor +public class ScalingDirective { + private final String profileName; + private final int setPoint; + private final long timestampEpochMillis; + private final Optional optDerivedFrom; + + public ScalingDirective(String profileName, int setPoint, long timestampEpochMillis) { + this(profileName, setPoint, timestampEpochMillis, Optional.empty()); + } + + public ScalingDirective(String profileName, int setPoint, long timestampEpochMillis, String basisProfileName, ProfileOverlay overlay) { + this(profileName, setPoint, timestampEpochMillis, Optional.of(new ProfileDerivation(basisProfileName, overlay))); + } + + public String renderName() { + return WorkforceProfiles.renderName(this.profileName); + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java new file mode 100644 index 00000000000..4dbbc065b78 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import lombok.extern.slf4j.Slf4j; + + +/** + * parse {@link ScalingDirective}s with syntax of the form: + * TIMESTAMP '.' WORKER_NAME '=' SETPOINT [ ( ',' | ';' ) WORKER_NAME ( '+(' KV_PAIR (*SEP* KV_PAIR)* ')' | '-( KEY (*SEP* KEY* ')' ) ] + * where *SEP* is either ',' or ';' (whichever did follow SETPOINT) + * the first form with '+' is an "adding" (upsert) overlay, the second form with '-' is a removing overlay + * allows for URL-encoded values in the KV_PAIRs and whitespace around any token + * + 1728435970.my_profile=24 + 1728436821.=24 + 1728436828.baseline()=24 + + 1728439210.new_profile=16,bar+(a.b.c=7,l.m=sixteen) + 1728439223.new_profile=16;bar+(a.b.c=7;l.m=sixteen) + 1728460832.new_profile=16,bar+(a.b.c=7,l.m=sixteen%2C%20again) + + 1728436436.other_profile=9,my_profile-(x,y.z) + 1728436499.other_profile=9;my_profile-(x;y.z) + + 1728441200.plus_profile=16,+(a.b.c=7,l.m=sixteen) + 1728443640.plus_profile=16,baseline()+(a.b.c=7,l.m=sixteen) + + 1728448521.extra_profile=9,-(a.b, c.d) + 1728449978.extra_profile=9,baseline()-(a.b, c.d) +*/ +@Slf4j +public class ScalingDirectiveParser { + public static class MalformedDirectiveException extends IllegalArgumentException { + private final String directive; + public MalformedDirectiveException(String directive, String desc) { + super("error: " + desc + ", in ==>" + directive + "<=="); + this.directive = directive; + } + } + + private static final String DIRECTIVE_REGEX = "(?x) \\s* (\\d+) \\s* \\. \\s* (\\w* | baseline\\(\\)) \\s* = \\s* (\\d+) " + + "(?: \\s* ([;,]) \\s* (\\w* | baseline\\(\\)) \\s* (?: (\\+ \\s* \\( \\s* ([^)]*?) \\s* \\) ) | (- \\s* \\( \\s* ([^)]*?) \\s* \\) ) ) )? \\s*"; + + private static final String KEY_REGEX = "(\\w+(?:\\.\\w+)*)"; + private static final String KEY_VALUE_REGEX = KEY_REGEX + "\\s*=\\s*(.*)"; + private static final Pattern directivePattern = Pattern.compile(DIRECTIVE_REGEX); + private static final Pattern keyPattern = Pattern.compile(KEY_REGEX); + private static final Pattern keyValuePattern = Pattern.compile(KEY_VALUE_REGEX); + + private static final String BASELINE_ID = "baseline()"; + + public ScalingDirective parse(String directive) { + Matcher parsed = directivePattern.matcher(directive); + if (parsed.matches()) { + long timestamp = Long.parseLong(parsed.group(1)); + String profileId = parsed.group(2); + String profileName = identifyProfileName(profileId); + int setpoint = Integer.parseInt(parsed.group(3)); + Optional optDerivedFrom = Optional.empty(); + String overlayIntroSep = parsed.group(4); + if (overlayIntroSep != null) { + String basisProfileName = identifyProfileName(parsed.group(5)); + if (parsed.group(6) != null) { // '+' == adding + List additions = new ArrayList<>(); + String additionsStr = parsed.group(7); + if (!additionsStr.equals("")) { + for (String addStr : additionsStr.split("\\s*" + overlayIntroSep + "\\s*", -1)) { // (negative limit to disallow trailing empty strings) + Matcher keyValueParsed = keyValuePattern.matcher(addStr); + if (keyValueParsed.matches()) { + additions.add(new ProfileOverlay.KVPair(keyValueParsed.group(1), urlDecode(directive, keyValueParsed.group(2)))); + } else { + throw new MalformedDirectiveException(directive, "unable to parse key-value pair - {{" + addStr + "}}"); + } + } + } + optDerivedFrom = Optional.of(new ProfileDerivation(basisProfileName, new ProfileOverlay.Adding(additions))); + } else { // '-' == removing + List removalKeys = new ArrayList<>(); + String removalsStr = parsed.group(9); + if (!removalsStr.equals("")) { + for (String removeStr : removalsStr.split("\\s*" + overlayIntroSep + "\\s*", -1)) { // (negative limit to disallow trailing empty strings) + Matcher keyParsed = keyPattern.matcher(removeStr); + if (keyParsed.matches()) { + removalKeys.add(keyParsed.group(1)); + } else { + throw new MalformedDirectiveException(directive, "unable to parse key - {{" + removeStr + "}}"); + } + } + } + optDerivedFrom = Optional.of(new ProfileDerivation(basisProfileName, new ProfileOverlay.Removing(removalKeys))); + } + } + return new ScalingDirective(profileName, setpoint, timestamp, optDerivedFrom); + } else { + throw new MalformedDirectiveException(directive, "invalid syntax"); + } + } + + private static String identifyProfileName(String profileId) { + return profileId.equals(BASELINE_ID) ? WorkforceProfiles.BASELINE_NAME : profileId; + } + + private static String urlDecode(String directive, String s) { + try { + return java.net.URLDecoder.decode(s, "UTF-8"); + } catch (java.io.UnsupportedEncodingException e) { + throw new MalformedDirectiveException(directive, "unable to URL-decode - {{" + s + "}}"); + } + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java new file mode 100644 index 00000000000..018af44ff95 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.List; +import lombok.Data; + + +@Data +public class StaffingDeltas { + @Data + public static class ProfileDelta { + private final WorkerProfile profile; + private final int delta; + private final long setPointProvenanceEpochMillis; + + public boolean isUnchanged() { + return delta == 0; + } + } + + + private final List perProfileDeltas; +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java new file mode 100644 index 00000000000..df67eb47224 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import com.typesafe.config.Config; +import lombok.Data; + + +@Data +public class WorkerProfile { + private final String name; + private final Config config; +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java new file mode 100644 index 00000000000..7c61eb65c13 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import com.google.common.annotations.VisibleForTesting; +import java.util.List; +import java.util.Optional; +import java.util.function.Consumer; + +import com.typesafe.config.Config; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + + +@Slf4j +public class WorkforcePlan { + + public static class IllegalRevisionException extends Exception { + @Getter private final ScalingDirective directive; + private IllegalRevisionException(ScalingDirective directive, String msg) { + super(msg); + this.directive = directive; + } + + public static class OutdatedDirective extends IllegalRevisionException { + protected OutdatedDirective(ScalingDirective directive, long lastRevisionEpochMillis) { + super(directive, "directive for profile '" + directive.renderName() + "' precedes last revision at " + + lastRevisionEpochMillis + ": " + directive); + } + } + + public static class Redefinition extends IllegalRevisionException { + protected Redefinition(ScalingDirective directive, ProfileDerivation proposedDerivation) { + super(directive, "profile '" + directive.renderName() + "' already exists, so may not be redefined on the basis of '" + + proposedDerivation.renderName() + "': " + directive); + } + } + + public static class UnrecognizedProfile extends IllegalRevisionException { + protected UnrecognizedProfile(ScalingDirective directive) { + super(directive, "unrecognized profile reference '" + directive.renderName() + "': " + directive); + } + } + + public static class UnknownBasis extends IllegalRevisionException { + protected UnknownBasis(ScalingDirective directive, ProfileDerivation.UnknownBasisException ube) { + super(directive, "profile '" + directive.renderName() + "' may not be defined on the basis of an unknown profile '" + + WorkforceProfiles.renderName(ube.getName()) + "': " + directive); + } + } + } + + private final WorkforceProfiles profiles; + private final WorkforceStaffing staffing; + @Getter private volatile long lastRevisionEpochMillis; + + public WorkforcePlan(Config baselineConfig, int initialSetPoint) { + this.profiles = WorkforceProfiles.withBaseline(baselineConfig); + this.staffing = WorkforceStaffing.initialize(initialSetPoint); + this.lastRevisionEpochMillis = 0; + } + + public int getNumProfiles() { + return profiles.size(); + } + + public synchronized void revise(ScalingDirective directive) throws IllegalRevisionException { + String name = directive.getProfileName(); + if (this.lastRevisionEpochMillis >= directive.getTimestampEpochMillis()) { + throw new IllegalRevisionException.OutdatedDirective(directive, this.lastRevisionEpochMillis); + }; + Optional optExistingProfile = profiles.apply(name); + Optional optDerivation = directive.getOptDerivedFrom(); + if (optExistingProfile.isPresent() && optDerivation.isPresent()) { + throw new IllegalRevisionException.Redefinition(directive, optDerivation.get()); + } else if (!optExistingProfile.isPresent() && !optDerivation.isPresent()) { + throw new IllegalRevisionException.UnrecognizedProfile(directive); + } else { // [exclusive-or: either, but not both present] + if (optDerivation.isPresent()) { // define a new profile on the basis of another + try { + this.profiles.addProfile(new WorkerProfile(name, optDerivation.get().formulateConfig(this.profiles))); + } catch (ProfileDerivation.UnknownBasisException ube) { + throw new IllegalRevisionException.UnknownBasis(directive, ube); + } + } + // adjust the set-point now that either a new profile is defined OR the profile already existed + this.staffing.reviseStaffing(name, directive.getSetPoint(), directive.getTimestampEpochMillis()); + this.lastRevisionEpochMillis = directive.getTimestampEpochMillis(); + } + } + + /** atomic bulk revision + * + * !!!!requires sorted order of directives by timestamp!!!! + * + */ + public synchronized void reviseWhenNewer(List directives) { + reviseWhenNewer(directives, ire -> { log.warn("Failure: ", ire); }); + } + + public synchronized void reviseWhenNewer(List directives, Consumer illegalRevisionHandler) { + directives.stream().sequential() + .filter(directive -> directive.getTimestampEpochMillis() > this.lastRevisionEpochMillis) + .forEach(directive -> { + try { + revise(directive); + } catch (IllegalRevisionException ire) { + System.err.println("uh oh it's: " + ire); + illegalRevisionHandler.accept(ire); + } + }); + } + + /** @returns diff of {@link StaffingDeltas} of this, current {@link WorkforcePlan} against some `reference` {@link WorkforceStaffing} */ + public synchronized StaffingDeltas calcStaffingDeltas(WorkforceStaffing reference) { + return staffing.calcDeltas(reference, profiles); + } + + @VisibleForTesting + Optional peepStaffing(String profileName) { + return staffing.getStaffing(profileName); + } + + @VisibleForTesting + WorkerProfile peepProfile(String profileName) throws WorkforceProfiles.UnknownProfileException { + return profiles.getOrThrow(profileName); + } + + @VisibleForTesting + WorkerProfile peepBaselineProfile() throws WorkforceProfiles.UnknownProfileException { + return profiles.getOrThrow(WorkforceProfiles.BASELINE_NAME); + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java new file mode 100644 index 00000000000..b76c33562e2 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; + +import com.typesafe.config.Config; + + +public class WorkforceProfiles implements Function> { + public static final String BASELINE_NAME = ""; + public static final String BASELINE_NAME_RENDERING = "<>"; + + public static String renderName(String name) { + return name.equals(BASELINE_NAME) ? BASELINE_NAME_RENDERING : name; + } + + + public static class UnknownProfileException extends RuntimeException { + public UnknownProfileException(String profileName) { + super("named '" + WorkforceProfiles.renderName(profileName) + "'"); + } + } + + private final ConcurrentHashMap profileByName; + + private WorkforceProfiles() { + this.profileByName = new ConcurrentHashMap<>(); + } + + public static WorkforceProfiles withBaseline(Config baselineConfig) { + WorkforceProfiles profiles = new WorkforceProfiles(); + profiles.addProfile(new WorkerProfile(BASELINE_NAME, baselineConfig)); + return profiles; + } + + @Override + public Optional apply(String profileName) { + return Optional.ofNullable(profileByName.get(profileName)); + } + + public WorkerProfile getOrThrow(String profileName) { + WorkerProfile profile = profileByName.get(profileName); + if (profile != null) { + return profile; + } + throw new UnknownProfileException(profileName); + } + + public void addProfile(WorkerProfile profile) { + if (profileByName.putIfAbsent(profile.getName(), profile) != null) { + throw new RuntimeException("profile '" + WorkforceProfiles.renderName(profile.getName()) + "' already exists!"); + } + } + + public int size() { + return profileByName.size(); + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java new file mode 100644 index 00000000000..f54c8035170 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import com.google.common.annotations.VisibleForTesting; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import com.google.common.base.Preconditions; +import lombok.Data; + + +public class WorkforceStaffing { + public static long INITIALIZATION_PROVENANCE_EPOCH_MILLIS = 0L; + // CAUTION: sentinel value only for use with `StaffingDeltas.ProfileDelta` - NOT for use with `WorkforceStaffing::reviseStaffing`! + public static long UNKNOWN_PROVENANCE_EPOCH_MILLIS = -1L; + + @Data + private static class SetPoint { + private final int point; + private final long provenanceEpochMillis; // for debuggability + } + + + private final Map setPointByName; + + private WorkforceStaffing() { + this.setPointByName = new ConcurrentHashMap<>(); + } + + public static WorkforceStaffing initialize(int initialBaselineSetPoint) { + WorkforceStaffing staffing = new WorkforceStaffing(); + staffing.reviseStaffing(WorkforceProfiles.BASELINE_NAME, initialBaselineSetPoint, INITIALIZATION_PROVENANCE_EPOCH_MILLIS); + return staffing; + } + + @VisibleForTesting + public static WorkforceStaffing initializeStaffing(int initialBaselineSetPoint, Map initialSetPointsByProfileName) { + WorkforceStaffing staffing = initialize(initialBaselineSetPoint); + initialSetPointsByProfileName.forEach((profileName, setPoint) -> + staffing.reviseStaffing(profileName, setPoint, INITIALIZATION_PROVENANCE_EPOCH_MILLIS) + ); + return staffing; + } + + public Optional getStaffing(String profileName) { + return Optional.ofNullable(setPointByName.get(profileName)).map(SetPoint::getPoint); + } + + public void reviseStaffing(String profileName, int setPoint, long provenanceEpochMillis) { + Preconditions.checkArgument(setPoint >= 0, "set points must be non-negative: '" + profileName + "' had " + setPoint); + Preconditions.checkArgument(provenanceEpochMillis >= INITIALIZATION_PROVENANCE_EPOCH_MILLIS, + "provenanceEpochMillis must be non-negative: '" + profileName + "' had " + provenanceEpochMillis); + setPointByName.put(profileName, new SetPoint(setPoint, provenanceEpochMillis)); + } + + /** + * NOTE: so long as the same {@link WorkforcePlan} managed both this {@link WorkforceStaffing} and {@link WorkforceProfiles}, + * {@link WorkforceProfiles.UnknownProfileException} should NOT be possible. + */ + public synchronized StaffingDeltas calcDeltas(WorkforceStaffing reference, WorkforceProfiles profiles) { + Map frozenReferenceSetPointsByName = new HashMap<>(); // freeze entries for consistency amidst multiple traversals + reference.setPointByName.entrySet().forEach(entry -> frozenReferenceSetPointsByName.put(entry.getKey(), entry.getValue())); + // not expecting any profile earlier in `reference` to no longer be set... (but defensive coding nonetheless) + List profileDeltas = frozenReferenceSetPointsByName.entrySet().stream() + .filter(entry -> !this.setPointByName.containsKey(entry.getKey())) + .map(entry -> new StaffingDeltas.ProfileDelta(profiles.getOrThrow(entry.getKey()), 0 - entry.getValue().getPoint(), UNKNOWN_PROVENANCE_EPOCH_MILLIS)) + .collect(Collectors.toList()); + profileDeltas.addAll(this.setPointByName.entrySet().stream().map(entry -> { + Optional optEquivReferenceSetPoint = Optional.ofNullable(frozenReferenceSetPointsByName.get(entry.getKey())).map(SetPoint::getPoint); + return new StaffingDeltas.ProfileDelta( + profiles.getOrThrow(entry.getKey()), + entry.getValue().getPoint() - optEquivReferenceSetPoint.orElse(0), + entry.getValue().getProvenanceEpochMillis()); + } + ).filter(delta -> !delta.isUnchanged()) + .collect(Collectors.toList())); + return new StaffingDeltas(profileDeltas); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java new file mode 100644 index 00000000000..d87df961c54 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.Optional; +import java.util.function.Function; + +import com.google.common.collect.Lists; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; + +import org.testng.annotations.Test; +import org.testng.Assert; + + +public class ProfileDerivationTest { + + @Test + public void testFormulateConfigSuccess() throws ProfileDerivation.UnknownBasisException { + String basisProfileName = "testProfile"; + ProfileOverlay overlay = new ProfileOverlay.Adding(Lists.newArrayList(new ProfileOverlay.KVPair("key1", "value1B"))); + ProfileDerivation profileDerivation = new ProfileDerivation(basisProfileName, overlay); + + Function> basisResolver = name -> { + if (basisProfileName.equals(name)) { + Config config = ConfigFactory.parseString("key1=value1A, key2=value2"); + WorkerProfile profile = new WorkerProfile(basisProfileName, config); + return Optional.of(profile); + } + return Optional.empty(); + }; + + Config resultConfig = profileDerivation.formulateConfig(basisResolver); + Assert.assertEquals(resultConfig.getString("key1"), "value1B"); + Assert.assertEquals(resultConfig.getString("key2"), "value2"); + } + + public void testFormulateConfigUnknownBasis() { + String basisProfileName = "foo"; + try { + ProfileDerivation derivation = new ProfileDerivation(basisProfileName, null); + derivation.formulateConfig(ignore -> Optional.empty()); + Assert.fail("Expected UnknownBasisException"); + } catch (ProfileDerivation.UnknownBasisException ube) { + Assert.assertEquals(ube.getName(), basisProfileName); + } + } + + @Test + public void testRenderNameNonBaseline() { + String name = "testProfile"; + ProfileDerivation profileDerivation = new ProfileDerivation(name, null); + String renderedName = profileDerivation.renderName(); + Assert.assertEquals(renderedName, name); + } + + @Test + public void testRenderNameBaseline() { + ProfileDerivation profileDerivation = new ProfileDerivation(WorkforceProfiles.BASELINE_NAME, null); + String renderedName = profileDerivation.renderName(); + Assert.assertEquals(renderedName, WorkforceProfiles.BASELINE_NAME_RENDERING); + } +} \ No newline at end of file diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java new file mode 100644 index 00000000000..125487cde03 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import com.google.common.collect.Lists; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; + +import org.testng.annotations.Test; +import org.testng.Assert; + + +public class ProfileOverlayTest { + + @Test + public void testAddingApplyOverlay() { + Config config = ConfigFactory.parseString("key1=value1A, key4=value4"); + ProfileOverlay.Adding adding = new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("key1", "value1B"), new ProfileOverlay.KVPair("key2", "value2"))); + Config updatedConfig = adding.applyOverlay(config); + Assert.assertEquals(updatedConfig.getString("key1"), "value1B"); + Assert.assertEquals(updatedConfig.getString("key2"), "value2"); + Assert.assertEquals(updatedConfig.getString("key4"), "value4"); + } + + @Test + public void testRemovingApplyOverlay() { + Config config = ConfigFactory.parseString("key1=value1, key2=value2"); + ProfileOverlay.Removing removing = new ProfileOverlay.Removing(Lists.newArrayList("key1")); + Config updatedConfig = removing.applyOverlay(config); + Assert.assertFalse(updatedConfig.hasPath("key1")); + Assert.assertEquals(updatedConfig.getString("key2"), "value2"); + } + + @Test + public void testComboApplyOverlay() { + Config config = ConfigFactory.parseString("key1=value1, key2=value2, key3=value3"); + ProfileOverlay.Adding adding = new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("key4", "value4"), new ProfileOverlay.KVPair("key5", "value5"))); + ProfileOverlay.Removing removing = new ProfileOverlay.Removing(Lists.newArrayList("key2", "key4")); + ProfileOverlay.Combo combo = ProfileOverlay.Combo.normalize(adding, removing); + Config updatedConfig = combo.applyOverlay(config); + Assert.assertEquals(updatedConfig.getString("key1"), "value1"); + Assert.assertEquals(updatedConfig.hasPath("key2"), false); + Assert.assertEquals(updatedConfig.getString("key3"), "value3"); + Assert.assertEquals(updatedConfig.hasPath("key4"), false); + Assert.assertEquals(updatedConfig.getString("key5"), "value5"); + + // validate `Combo::normalize` works too: + Assert.assertEquals(combo.getAdding().getAdditionPairs().size(), 1); + Assert.assertEquals(combo.getAdding().getAdditionPairs().get(0), new ProfileOverlay.KVPair("key5", "value5")); + Assert.assertEquals(combo.getRemoving().getRemovalKeys().size(), 2); + Assert.assertEqualsNoOrder(combo.getRemoving().getRemovalKeys().toArray(), removing.getRemovalKeys().toArray()); + } + + @Test + public void testAddingOver() { + ProfileOverlay.Adding adding1 = new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("key1", "value1"), new ProfileOverlay.KVPair("key2", "value2A"))); + ProfileOverlay.Adding adding2 = new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("key2", "value2B"), new ProfileOverlay.KVPair("key3", "value3"))); + ProfileOverlay result = adding1.over(adding2); + Config config = result.applyOverlay(ConfigFactory.empty()); + Assert.assertEquals(config.getString("key1"), "value1"); + Assert.assertEquals(config.getString("key2"), "value2A"); + Assert.assertEquals(config.getString("key3"), "value3"); + } + + @Test + public void testRemovingOver() { + ProfileOverlay.Removing removing1 = new ProfileOverlay.Removing(Lists.newArrayList("key1", "key2")); + ProfileOverlay.Removing removing2 = new ProfileOverlay.Removing(Lists.newArrayList("key2", "key3")); + ProfileOverlay result = removing1.over(removing2); + Assert.assertTrue(result instanceof ProfileOverlay.Removing); + ProfileOverlay.Removing removingResult = (ProfileOverlay.Removing) result; + Assert.assertEqualsNoOrder(removingResult.getRemovalKeys().toArray(), new String[]{"key1", "key2", "key3"}); + + Config config = + result.applyOverlay(ConfigFactory.parseString("key1=value1, key2=value2, key3=value3, key4=value4")); + Assert.assertFalse(config.hasPath("key1")); + Assert.assertFalse(config.hasPath("key2")); + Assert.assertFalse(config.hasPath("key3")); + Assert.assertTrue(config.hasPath("key4")); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java new file mode 100644 index 00000000000..282d7a52987 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.Optional; +import com.google.common.collect.Lists; + +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import org.testng.Assert; + + +public class ScalingDirectiveParserTest { + + private final ScalingDirectiveParser parser = new ScalingDirectiveParser(); + + @Test + public void parseSimpleDirective() { + ScalingDirective sd = parser.parse("1728435970.my_profile=24"); + Assert.assertEquals(sd.getTimestampEpochMillis(), 1728435970L); + Assert.assertEquals(sd.getProfileName(), "my_profile"); + Assert.assertEquals(sd.getSetPoint(), 24); + Assert.assertFalse(sd.getOptDerivedFrom().isPresent()); + } + + @Test + public void parseUnnamedBaselineProfile() { + ScalingDirective sd = parser.parse("1728436821.=12"); + Assert.assertEquals(sd.getTimestampEpochMillis(), 1728436821L); + Assert.assertEquals(sd.getProfileName(), WorkforceProfiles.BASELINE_NAME); + Assert.assertEquals(sd.getSetPoint(), 12); + Assert.assertFalse(sd.getOptDerivedFrom().isPresent()); + } + + @Test + public void parseBaselineProfile() { + ScalingDirective sd = parser.parse("1728436828.baseline()=6"); + Assert.assertEquals(sd, new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 6, 1728436828L, Optional.empty())); + } + + @Test + public void parseAddingOverlayWithCommaSep() { + ScalingDirective sd = parser.parse("1728439210.new_profile=16,bar+(a.b.c=7,l.m=sixteen)"); + Assert.assertEquals(sd.getTimestampEpochMillis(), 1728439210L); + Assert.assertEquals(sd.getProfileName(), "new_profile"); + Assert.assertEquals(sd.getSetPoint(), 16); + Assert.assertTrue(sd.getOptDerivedFrom().isPresent()); + ProfileDerivation derivation = sd.getOptDerivedFrom().get(); + Assert.assertEquals(derivation.getBasisProfileName(), "bar"); + Assert.assertEquals(derivation.getOverlay(), new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m", "sixteen")))); + } + + @Test + public void parseAddingOverlayWithSemicolonSep() { + ScalingDirective sd = parser.parse("1728439223.new_profile=32;baz+( a.b.c=7 ; l.m.n.o=sixteen )"); + Assert.assertEquals(sd, new ScalingDirective("new_profile", 32, 1728439223L, "baz", new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m.n.o", "sixteen"))))); + } + + @Test + public void parseAddingOverlayWithCommaSepUrlEncoded() { + ScalingDirective sd = parser.parse("1728460832.new_profile=16,baa+(a.b.c=7,l.m=sixteen%2C%20again)"); + Assert.assertEquals(sd, new ScalingDirective("new_profile", 16, 1728460832L, "baa", new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m", "sixteen, again"))))); + } + + @Test + public void parseRemovingOverlayWithCommaSep() { + ScalingDirective sd = parser.parse("1728436436.other_profile=9,my_profile-( x , y.z )"); + Assert.assertEquals(sd.getTimestampEpochMillis(), 1728436436L); + Assert.assertEquals(sd.getProfileName(), "other_profile"); + Assert.assertEquals(sd.getSetPoint(), 9); + Assert.assertTrue(sd.getOptDerivedFrom().isPresent()); + ProfileDerivation derivation = sd.getOptDerivedFrom().get(); + Assert.assertEquals(derivation.getBasisProfileName(), "my_profile"); + Assert.assertEquals(derivation.getOverlay(), new ProfileOverlay.Removing(Lists.newArrayList("x", "y.z"))); + } + + @Test + public void parseRemovingOverlayWithSemicolonSep() { + ScalingDirective sd = parser.parse("1728436499.other_profile=9;my_profile-(x.y;z.z)"); + Assert.assertEquals(sd, new ScalingDirective("other_profile", 9, 1728436499L, "my_profile", + new ProfileOverlay.Removing(Lists.newArrayList("x.y", "z.z")))); + } + + @Test + public void parseAddingOverlayWithWhitespace() { + ScalingDirective sd = parser.parse(" 1728998877 . another = 999 ; wow + ( t.r = jump%20 ; cb.az = foo%20#%20111 ) "); + Assert.assertEquals(sd, new ScalingDirective("another", 999, 1728998877L, "wow", new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("t.r", "jump "), + new ProfileOverlay.KVPair("cb.az", "foo # 111"))))); + } + + @Test + public void parseRemovingOverlayWithWhitespace() { + ScalingDirective sd = parser.parse(" 1728334455 . also = 77 , really - ( t.r , cb.az ) "); + Assert.assertEquals(sd, new ScalingDirective("also", 77, 1728334455L, "really", + new ProfileOverlay.Removing(Lists.newArrayList("t.r", "cb.az")))); + } + + @Test + public void parseAddingOverlayWithUnnamedBaselineProfile() { + ScalingDirective sd = parser.parse("1728441200.plus_profile=16,+(q.r.s=four,l.m=16)"); + Assert.assertEquals(sd, new ScalingDirective("plus_profile", 16, 1728441200L, WorkforceProfiles.BASELINE_NAME, + new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("q.r.s", "four"), new ProfileOverlay.KVPair("l.m", "16"))))); + } + + @Test + public void parseAddingOverlayWithBaselineProfile() { + ScalingDirective sd = parser.parse("1728443640.plus_profile=16,baseline()+(q.r=five,l.m=12)"); + Assert.assertEquals(sd, new ScalingDirective("plus_profile", 16, 1728443640L, WorkforceProfiles.BASELINE_NAME, + new ProfileOverlay.Adding( + Lists.newArrayList(new ProfileOverlay.KVPair("q.r", "five"), new ProfileOverlay.KVPair("l.m", "12"))))); + } + + @Test + public void parseRemovingOverlayWithUnnamedBaselineProfile() { + ScalingDirective sd = parser.parse("1728448521.extra_profile=0,-(a.b, c.d)"); + Assert.assertEquals(sd, new ScalingDirective("extra_profile", 0, 1728448521L, WorkforceProfiles.BASELINE_NAME, + new ProfileOverlay.Removing(Lists.newArrayList("a.b", "c.d")))); + } + + @Test + public void parseRemovingOverlayWithBaselineProfile() { + ScalingDirective sd = parser.parse("4.extra_profile=9,baseline()-(a.b, c.d)"); + Assert.assertEquals(sd, new ScalingDirective("extra_profile", 9, 4L, WorkforceProfiles.BASELINE_NAME, + new ProfileOverlay.Removing(Lists.newArrayList("a.b", "c.d")))); + } + + + @DataProvider(name = "funkyButValidDirectives") + public String[][] validDirectives() { + return new String[][]{ + // null overlay upon unnamed baseline profile: + {"1728435970.my_profile=24,+()"}, + {"1728435970.my_profile=24,-()"}, + {"1728435970.my_profile=24;+()"}, + {"1728435970.my_profile=24;-()"}, + + // null overlay upon named profile: + {"1728435970.my_profile=24,foo+()"}, + {"1728435970.my_profile=24,foo-()"}, + {"1728435970.my_profile=24;foo+()"}, + {"1728435970.my_profile=24;foo-()"}, + + // seemingly separator mismatch, but in fact the NOT-separator is part of the value (e.g. a="7;m=sixteen"): + { "1728439210.new_profile=16,bar+(a=7;m=sixteen)" }, + { "1728439210.new_profile=16;bar+(a=7,m=sixteen)" }, + { "1728439210.new_profile=16,bar+(a=7;)" }, + { "1728439210.new_profile=16;bar+(a=7,)" } + }; + } + + @Test( + expectedExceptions = {}, + dataProvider = "funkyButValidDirectives" + ) + public void parseValidDirectives(String directive) { + Assert.assertNotNull(parser.parse(directive)); + } + + + @DataProvider(name = "invalidDirectives") + public String[][] invalidDirectives() { + return new String[][] { + // invalid values: + { "invalid_timestamp.my_profile=24" }, + { "1728435970.my_profile=invalid_setpoint" }, + { "1728435970.my_profile=-15" }, + + // incomplete/fragments: + { "1728435970.my_profile=24," }, + { "1728435970.my_profile=24;" }, + { "1728435970.my_profile=24,+" }, + { "1728435970.my_profile=24,-" }, + { "1728435970.my_profile=24,foo+" }, + { "1728435970.my_profile=24,foo-" }, + { "1728435970.my_profile=24,foo+a=7" }, + { "1728435970.my_profile=24,foo-x" }, + + // adding: invalid set-point + missing token examples: + { "1728439210.new_profile=-6,bar+(a=7,m=sixteen)" }, + { "1728439210.new_profile=16,bar+(a=7,m=sixteen" }, + { "1728439210.new_profile=16,bar+a=7,m=sixteen)" }, + + // adding: key, instead of key-value pair: + { "1728439210.new_profile=16,bar+(a=7,m)" }, + { "1728439210.new_profile=16,bar+(a,m)" }, + + // adding: superfluous separator or used instead as a terminator: + { "1728439210.new_profile=16,bar+(,)" }, + { "1728439210.new_profile=16;bar+(;)" }, + { "1728439210.new_profile=16,bar+(,,)" }, + { "1728439210.new_profile=16;bar+(;;)" }, + { "1728439210.new_profile=16,bar+(a=7,)" }, + { "1728439210.new_profile=16;bar+(a=7;)" }, + + // removing: invalid set-point + missing token examples: + { "1728436436.other_profile=-9,my_profile-(x)" }, + { "1728436436.other_profile=69,my_profile-(x" }, + { "1728436436.other_profile=69,my_profile-x)" }, + + // removing: key-value pair instead of key: + { "1728436436.other_profile=69,my_profile-(x=y,z)" }, + { "1728436436.other_profile=69,my_profile-(x=y,z=1)" }, + + // removing: superfluous separator or used instead as a terminator: + { "1728436436.other_profile=69,my_profile-(,)" }, + { "1728436436.other_profile=69;my_profile-(;)" }, + { "1728436436.other_profile=69,my_profile-(,,)" }, + { "1728436436.other_profile=69;my_profile-(;;)" }, + { "1728436436.other_profile=69,my_profile-(x,)" }, + { "1728436436.other_profile=69;my_profile-(x;)" }, + + // removing: seemingly separator mismatch, but in fact the NOT-separator is illegal in a key (e.g. "x;y"): + { "1728436436.other_profile=69,my_profile-(x;y)" }, + { "1728436436.other_profile=69;my_profile-(x,y)" }, + { "1728436436.other_profile=69,my_profile-(x;)" }, + { "1728436436.other_profile=69;my_profile-(x,)" } + }; + } + + @Test( + expectedExceptions = ScalingDirectiveParser.MalformedDirectiveException.class, + dataProvider = "invalidDirectives" + ) + public void parseInvalidDirectives(String directive) { + parser.parse(directive); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java new file mode 100644 index 00000000000..838964aa4fc --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; + +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import org.testng.Assert; + + +public class WorkforcePlanTest { + private Config baselineConfig = ConfigFactory.parseString("key1=value1, key2=value2"); + private final int initialBaselineSetPoint = 10; + private WorkforcePlan plan; + + @BeforeMethod + public void setUp() { + plan = new WorkforcePlan(baselineConfig, initialBaselineSetPoint); + } + + private static ScalingDirective createNewProfileDirective(String profileName, int setPoint, long epochMillis, String basisProfileName) { + return new ScalingDirective(profileName, setPoint, epochMillis, Optional.of( + new ProfileDerivation(basisProfileName, new ProfileOverlay.Adding(Lists.newArrayList( + new ProfileOverlay.KVPair("key1", "new_value"), + new ProfileOverlay.KVPair("key4", "value4")))))); + } + + @Test + public void reviseWithValidReSetPoint() throws WorkforcePlan.IllegalRevisionException { + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7,10000L)); + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 1,20000L)); + Assert.assertEquals(plan.getLastRevisionEpochMillis(), 20000L); + Assert.assertEquals(plan.getNumProfiles(), 1); + } + + @Test + public void reviseWithValidDerivation() throws WorkforcePlan.IllegalRevisionException { + Assert.assertEquals(plan.getLastRevisionEpochMillis(), WorkforceStaffing.INITIALIZATION_PROVENANCE_EPOCH_MILLIS); + Assert.assertEquals(plan.getNumProfiles(), 1); + ScalingDirective directive = createNewProfileDirective("new_profile", 5,10000L, WorkforceProfiles.BASELINE_NAME); + plan.revise(directive); + + Assert.assertEquals(plan.getLastRevisionEpochMillis(), 10000L); + Assert.assertEquals(plan.getNumProfiles(), 2); + Config expectedConfig = ConfigFactory.parseString("key1=new_value, key2=value2, key4=value4"); + Assert.assertEquals(plan.peepProfile("new_profile").getConfig(), expectedConfig); + } + + @Test + public void reviseWhenNewerIgnoresOutOfOrderDirectives() throws WorkforcePlan.IllegalRevisionException { + AtomicInteger numErrors = new AtomicInteger(0); + Assert.assertEquals(plan.getLastRevisionEpochMillis(), WorkforceStaffing.INITIALIZATION_PROVENANCE_EPOCH_MILLIS); + Assert.assertEquals(plan.getNumProfiles(), 1); + plan.reviseWhenNewer(Lists.newArrayList( + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 2,100L), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 3,500L), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 4,200L), + createNewProfileDirective("new_profile", 5,400L, WorkforceProfiles.BASELINE_NAME), + // NOTE: the second attempt at derivation is NOT judged a duplicate, as the outdated timestamp of first attempt (above) meant it was ignored! + createNewProfileDirective("new_profile", 6,600L, WorkforceProfiles.BASELINE_NAME), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7,800L), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8,700L) + ), failure -> numErrors.incrementAndGet()); + + Assert.assertEquals(plan.getLastRevisionEpochMillis(), 800L); + Assert.assertEquals(plan.getNumProfiles(), 2); + Assert.assertEquals(numErrors.get(), 0); + Assert.assertEquals(plan.peepStaffing(WorkforceProfiles.BASELINE_NAME), Optional.of(7), WorkforceProfiles.BASELINE_NAME_RENDERING); + Assert.assertEquals(plan.peepStaffing("new_profile"), Optional.of(6), "new_profile"); + } + + @Test + public void reviseWhenNewerSwallowsErrors() throws WorkforcePlan.IllegalRevisionException { + AtomicInteger numErrors = new AtomicInteger(0); + plan.reviseWhenNewer(Lists.newArrayList( + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 1,100L), + // (1) error: `UnrecognizedProfile` + new ScalingDirective("unknown_profile", 2,250L), + createNewProfileDirective("new_profile", 3,200L, WorkforceProfiles.BASELINE_NAME), + // (2) error: `Redefinition` + createNewProfileDirective("new_profile", 4,450L, WorkforceProfiles.BASELINE_NAME), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 5,300L), + // (3) error: `UnknownBasis` + createNewProfileDirective("other_profile", 6,550L, "never_defined"), + new ScalingDirective("new_profile", 7,400L), + // ignored: out-of-order timestamp (not an error... see: `reviseWhenNewerIgnoresOutOfOrderDirectives`) + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8,350L), + createNewProfileDirective("another", 9,500L, "new_profile") + ), failure -> numErrors.incrementAndGet()); + + Assert.assertEquals(plan.getLastRevisionEpochMillis(), 500L); + Assert.assertEquals(plan.getNumProfiles(), 3); + Assert.assertEquals(numErrors.get(), 3); + Assert.assertEquals(plan.peepStaffing(WorkforceProfiles.BASELINE_NAME), Optional.of(5), WorkforceProfiles.BASELINE_NAME_RENDERING); + Assert.assertEquals(plan.peepStaffing("new_profile"), Optional.of(7), "new_profile"); + Assert.assertEquals(plan.peepStaffing("another"), Optional.of(9), "another"); + } + + @Test + public void calcStaffingDeltas() throws WorkforcePlan.IllegalRevisionException { + plan.revise(createNewProfileDirective("new_profile", 3,10L, WorkforceProfiles.BASELINE_NAME)); + plan.revise(createNewProfileDirective("other_profile", 8,20L, "new_profile")); + plan.revise(createNewProfileDirective("another", 7,30L, "new_profile")); + plan.revise(new ScalingDirective("new_profile", 5,40L)); + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 6,50L)); + plan.revise(new ScalingDirective("another", 4,60L)); + + Assert.assertEquals(plan.getLastRevisionEpochMillis(), 60L); + Assert.assertEquals(plan.getNumProfiles(), 4); + Assert.assertEquals(plan.peepStaffing(WorkforceProfiles.BASELINE_NAME), Optional.of(6), WorkforceProfiles.BASELINE_NAME_RENDERING); + Assert.assertEquals(plan.peepStaffing("new_profile"), Optional.of(5), "new_profile"); + Assert.assertEquals(plan.peepStaffing("another"), Optional.of(4), "another"); + Assert.assertEquals(plan.peepStaffing("other_profile"), Optional.of(8), "other_profile"); + + WorkforceStaffing referenceStaffing = WorkforceStaffing.initializeStaffing(100, ImmutableMap.of( + WorkforceProfiles.BASELINE_NAME, 100, + "new_profile", 1, + // not initialized - "another" + "other_profile", 8 + )); + StaffingDeltas deltas = plan.calcStaffingDeltas(referenceStaffing); + Assert.assertEquals(deltas.getPerProfileDeltas().size(), 3); + deltas.getPerProfileDeltas().forEach(delta -> { + switch (delta.getProfile().getName()) { + case WorkforceProfiles.BASELINE_NAME: + Assert.assertEquals(delta.getDelta(), -94); + Assert.assertEquals(delta.getSetPointProvenanceEpochMillis(), 50L); + break; + case "new_profile": + Assert.assertEquals(delta.getDelta(), 4); + Assert.assertEquals(delta.getSetPointProvenanceEpochMillis(), 40L); + break; + case "another": + Assert.assertEquals(delta.getDelta(), 4); + Assert.assertEquals(delta.getSetPointProvenanceEpochMillis(), 60L); + break; + case "other_profile": // NOTE: should NOT be present (since delta == 0)! + default: + Assert.fail("Unexpected profile: " + delta.getProfile().getName()); + } + }); + } + + @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.OutdatedDirective.class) + public void reviseWithOutdatedDirective() throws WorkforcePlan.IllegalRevisionException { + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7,30000L)); + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 12,8000L)); + } + + @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.UnrecognizedProfile.class) + public void reviseWithUnrecognizedProfileDirective() throws WorkforcePlan.IllegalRevisionException { + plan.revise(new ScalingDirective("unknown_profile", 7,10000L)); + } + + @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.Redefinition.class) + public void reviseWithRedefinitionDirective() throws WorkforcePlan.IllegalRevisionException { + plan.revise(createNewProfileDirective("new_profile", 5,10000L, WorkforceProfiles.BASELINE_NAME)); + plan.revise(createNewProfileDirective("new_profile", 9,20000L, WorkforceProfiles.BASELINE_NAME)); + } + + @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.UnknownBasis.class) + public void reviseWithUnknownBasisDirective() throws WorkforcePlan.IllegalRevisionException { + plan.revise(createNewProfileDirective("new_profile", 5,10000L, "never_defined")); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffingTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffingTest.java new file mode 100644 index 00000000000..e34673195c3 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffingTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynscale; + +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableMap; + +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import org.testng.Assert; + +import static org.mockito.ArgumentMatchers.anyString; + + +public class WorkforceStaffingTest { + + @Mock private WorkforceProfiles profiles; + + @BeforeMethod + public void setUp() { + MockitoAnnotations.openMocks(this); + Mockito.when(profiles.getOrThrow(anyString())).thenAnswer(invocation -> + new WorkerProfile(invocation.getArgument(0), null)); + } + + @Test + public void initializeShouldSetInitialBaselineSetPoint() { + int initialBaselineSetPoint = 5; + WorkforceStaffing staffing = WorkforceStaffing.initialize(initialBaselineSetPoint); + Assert.assertEquals(staffing.getStaffing(WorkforceProfiles.BASELINE_NAME), Optional.of(initialBaselineSetPoint)); + } + + @Test + public void reviseStaffingShouldUpdateSetPoint() { + String profileName = "testProfile"; + WorkforceStaffing staffing = WorkforceStaffing.initialize(0); + staffing.reviseStaffing(profileName, 10, 1000L); + Assert.assertEquals(staffing.getStaffing(profileName), Optional.of(10)); + + staffing.reviseStaffing(profileName, 17, 2000L); + Assert.assertEquals(staffing.getStaffing(profileName), Optional.of(17)); + } + + @Test + public void calcDeltasShouldReturnCorrectDeltas() { + String subsequentlyUnreferencedProfileName = "unreferenced"; + String newlyAddedProfileName = "added"; + String heldSteadyProfileName = "steady"; + WorkforceStaffing currentStaffing = WorkforceStaffing.initialize(5); + currentStaffing.reviseStaffing(subsequentlyUnreferencedProfileName, 3, 1000L); + currentStaffing.reviseStaffing(heldSteadyProfileName, 9, 2000L); + + WorkforceStaffing improvedStaffing = WorkforceStaffing.initialize(7); + improvedStaffing.reviseStaffing(newlyAddedProfileName, 10, 3000L); + improvedStaffing.reviseStaffing(heldSteadyProfileName, 9, 4000L); + + StaffingDeltas deltas = improvedStaffing.calcDeltas(currentStaffing, profiles); + Assert.assertEquals(deltas.getPerProfileDeltas().size(), 3); + // validate every delta + Map deltaByProfileName = deltas.getPerProfileDeltas().stream() + .collect(Collectors.toMap(delta -> delta.getProfile().getName(), StaffingDeltas.ProfileDelta::getDelta)); + ImmutableMap expectedDeltaByProfileName = ImmutableMap.of( + WorkforceProfiles.BASELINE_NAME, 2, + subsequentlyUnreferencedProfileName, -3, + // NOTE: NOT present (when delta == 0)! + // heldSteadyProfileName, 0, + newlyAddedProfileName, 10 + ); + Assert.assertEqualsNoOrder(deltaByProfileName.keySet().toArray(), expectedDeltaByProfileName.keySet().toArray()); + Assert.assertEquals(deltaByProfileName.get(WorkforceProfiles.BASELINE_NAME), expectedDeltaByProfileName.get(WorkforceProfiles.BASELINE_NAME)); + Assert.assertEquals(deltaByProfileName.get(subsequentlyUnreferencedProfileName), expectedDeltaByProfileName.get(subsequentlyUnreferencedProfileName)); + Assert.assertEquals(deltaByProfileName.get(newlyAddedProfileName), expectedDeltaByProfileName.get(newlyAddedProfileName)); + } +} From bdd3afcddf3a4d0804a14600917963d8edcb10ab Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Mon, 21 Oct 2024 11:35:57 -0700 Subject: [PATCH 02/11] change package name from `dynscale` to `dynamic` and add a few comments --- .../{dynscale => dynamic}/ProfileDerivation.java | 2 +- .../temporal/{dynscale => dynamic}/ProfileOverlay.java | 4 +++- .../{dynscale => dynamic}/ScalingDirective.java | 2 +- .../{dynscale => dynamic}/ScalingDirectiveParser.java | 2 +- .../temporal/{dynscale => dynamic}/StaffingDeltas.java | 2 +- .../temporal/{dynscale => dynamic}/WorkerProfile.java | 2 +- .../temporal/{dynscale => dynamic}/WorkforcePlan.java | 3 ++- .../{dynscale => dynamic}/WorkforceProfiles.java | 2 +- .../{dynscale => dynamic}/WorkforceStaffing.java | 6 +++--- .../{dynscale => dynamic}/ProfileDerivationTest.java | 2 +- .../{dynscale => dynamic}/ProfileOverlayTest.java | 2 +- .../ScalingDirectiveParserTest.java | 10 ++++++---- .../{dynscale => dynamic}/WorkforcePlanTest.java | 2 +- .../{dynscale => dynamic}/WorkforceStaffingTest.java | 2 +- 14 files changed, 24 insertions(+), 19 deletions(-) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/ProfileDerivation.java (97%) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/ProfileOverlay.java (98%) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/ScalingDirective.java (97%) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/ScalingDirectiveParser.java (99%) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/StaffingDeltas.java (96%) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/WorkerProfile.java (95%) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/WorkforcePlan.java (96%) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/WorkforceProfiles.java (98%) rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/{dynscale => dynamic}/WorkforceStaffing.java (96%) rename gobblin-temporal/src/test/java/org/apache/gobblin/temporal/{dynscale => dynamic}/ProfileDerivationTest.java (98%) rename gobblin-temporal/src/test/java/org/apache/gobblin/temporal/{dynscale => dynamic}/ProfileOverlayTest.java (99%) rename gobblin-temporal/src/test/java/org/apache/gobblin/temporal/{dynscale => dynamic}/ScalingDirectiveParserTest.java (95%) rename gobblin-temporal/src/test/java/org/apache/gobblin/temporal/{dynscale => dynamic}/WorkforcePlanTest.java (99%) rename gobblin-temporal/src/test/java/org/apache/gobblin/temporal/{dynscale => dynamic}/WorkforceStaffingTest.java (98%) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java similarity index 97% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java index 0e192c2c784..013e9139581 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.Optional; import java.util.function.Function; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileOverlay.java similarity index 98% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileOverlay.java index ed36ee9c16f..45434d74fc4 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileOverlay.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.ArrayList; import java.util.HashSet; @@ -44,6 +44,7 @@ class KVPair { @Data + // TODO: variadic ctor/factory class Adding implements ProfileOverlay { private final List additionPairs; @@ -77,6 +78,7 @@ public ProfileOverlay over(ProfileOverlay other) { @Data + // TODO: variadic ctor/factory class Removing implements ProfileOverlay { private final List removalKeys; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java similarity index 97% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java index 77d9b483c9a..5a4da3d4d38 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.Optional; import lombok.Data; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java similarity index 99% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java index 4dbbc065b78..973d078b6ff 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.ArrayList; import java.util.List; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/StaffingDeltas.java similarity index 96% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/StaffingDeltas.java index 018af44ff95..0cf0a8a5a94 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/StaffingDeltas.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.List; import lombok.Data; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkerProfile.java similarity index 95% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkerProfile.java index df67eb47224..385bf7e6d5f 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkerProfile.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import com.typesafe.config.Config; import lombok.Data; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java similarity index 96% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java index 7c61eb65c13..33c736b9f3b 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import com.google.common.annotations.VisibleForTesting; import java.util.List; @@ -98,6 +98,7 @@ public synchronized void revise(ScalingDirective directive) throws IllegalRevisi throw new IllegalRevisionException.UnknownBasis(directive, ube); } } + // TODO - make idempotent, as re-attempts after failure between `addProfile` and `reviseStaffing` would fail with `IllegalRevisionException.Redefinition` // adjust the set-point now that either a new profile is defined OR the profile already existed this.staffing.reviseStaffing(name, directive.getSetPoint(), directive.getTimestampEpochMillis()); this.lastRevisionEpochMillis = directive.getTimestampEpochMillis(); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java similarity index 98% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java index b76c33562e2..dbe9a597306 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java similarity index 96% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java index f54c8035170..b9f66acd970 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import com.google.common.annotations.VisibleForTesting; import java.util.HashMap; @@ -30,9 +30,9 @@ public class WorkforceStaffing { - public static long INITIALIZATION_PROVENANCE_EPOCH_MILLIS = 0L; + public static final long INITIALIZATION_PROVENANCE_EPOCH_MILLIS = 0L; // CAUTION: sentinel value only for use with `StaffingDeltas.ProfileDelta` - NOT for use with `WorkforceStaffing::reviseStaffing`! - public static long UNKNOWN_PROVENANCE_EPOCH_MILLIS = -1L; + public static final long UNKNOWN_PROVENANCE_EPOCH_MILLIS = -1L; @Data private static class SetPoint { diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileDerivationTest.java similarity index 98% rename from gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java rename to gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileDerivationTest.java index d87df961c54..c10636ed8a0 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileDerivationTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.Optional; import java.util.function.Function; diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileOverlayTest.java similarity index 99% rename from gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java rename to gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileOverlayTest.java index 125487cde03..ab9c70ef216 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileOverlayTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import com.google.common.collect.Lists; import com.typesafe.config.Config; diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java similarity index 95% rename from gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java rename to gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java index 282d7a52987..574f4966411 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.Optional; import com.google.common.collect.Lists; @@ -148,7 +148,7 @@ public void parseRemovingOverlayWithBaselineProfile() { @DataProvider(name = "funkyButValidDirectives") public String[][] validDirectives() { return new String[][]{ - // null overlay upon unnamed baseline profile: + // null overlay upon unnamed baseline profile (null overlay functions as the 'identity element'): {"1728435970.my_profile=24,+()"}, {"1728435970.my_profile=24,-()"}, {"1728435970.my_profile=24;+()"}, @@ -165,6 +165,8 @@ public String[][] validDirectives() { { "1728439210.new_profile=16;bar+(a=7,m=sixteen)" }, { "1728439210.new_profile=16,bar+(a=7;)" }, { "1728439210.new_profile=16;bar+(a=7,)" } + + // NOTE: unlike Adding, separator mismatch causes failure with the Removing overlay, because the NOT-separator is illegal in a key }; } @@ -204,7 +206,7 @@ public String[][] invalidDirectives() { { "1728439210.new_profile=16,bar+(a=7,m)" }, { "1728439210.new_profile=16,bar+(a,m)" }, - // adding: superfluous separator or used instead as a terminator: + // adding: superfluous separator or used incorrectly as a terminator: { "1728439210.new_profile=16,bar+(,)" }, { "1728439210.new_profile=16;bar+(;)" }, { "1728439210.new_profile=16,bar+(,,)" }, @@ -221,7 +223,7 @@ public String[][] invalidDirectives() { { "1728436436.other_profile=69,my_profile-(x=y,z)" }, { "1728436436.other_profile=69,my_profile-(x=y,z=1)" }, - // removing: superfluous separator or used instead as a terminator: + // removing: superfluous separator or used incorrectly as a terminator: { "1728436436.other_profile=69,my_profile-(,)" }, { "1728436436.other_profile=69;my_profile-(;)" }, { "1728436436.other_profile=69,my_profile-(,,)" }, diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java similarity index 99% rename from gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java rename to gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java index 838964aa4fc..8bc24897e9b 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffingTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffingTest.java similarity index 98% rename from gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffingTest.java rename to gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffingTest.java index e34673195c3..03e02e3248c 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffingTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffingTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.dynscale; +package org.apache.gobblin.temporal.dynamic; import java.util.Map; import java.util.Optional; From fd1dd360dfea8b78af5c527b90b94b0bb57c4d4c Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Mon, 21 Oct 2024 22:24:18 -0700 Subject: [PATCH 03/11] Add `ScalingDirectiveParser::asString` --- .../AbstractEagerFsDirBackedWorkload.java | 3 +- .../impl/ExecuteGobblinWorkflowImpl.java | 6 +-- .../dynamic/ScalingDirectiveParser.java | 41 ++++++++++++++++++- .../dynamic/ScalingDirectiveParserTest.java | 38 +++++++++++++---- 4 files changed, 72 insertions(+), 16 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java index f6b6e05f104..eea5d14ebe7 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java @@ -54,7 +54,7 @@ */ @lombok.NoArgsConstructor // IMPORTANT: for jackson (de)serialization @lombok.RequiredArgsConstructor -@lombok.ToString(exclude = { "stateConfig", "cachedWorkItems" }) +@lombok.ToString(exclude = { "cachedWorkItems" }) @Slf4j public abstract class AbstractEagerFsDirBackedWorkload implements Workload, FileSystemApt { @@ -64,7 +64,6 @@ public abstract class AbstractEagerFsDirBackedWorkload implements Wor // Cannot construct instance of `org.apache.hadoop.fs.Path` (although at least one Creator exists): // cannot deserialize from Object value (no delegate- or property-based Creator) @NonNull private String fsDir; - @Getter(AccessLevel.PROTECTED) @Setter(AccessLevel.PROTECTED) private transient volatile WORK_ITEM[] cachedWorkItems = null; @Override diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java index 8eab3ef0bd7..2aa2a7e6495 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java @@ -122,8 +122,7 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event throw ApplicationFailure.newNonRetryableFailureWithCause( String.format("Failed Gobblin job %s", jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY)), e.getClass().getName(), - e, - null + e ); } finally { // TODO: Cleanup WorkUnit/Taskstate Directory for jobs cancelled mid flight @@ -140,8 +139,7 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event throw ApplicationFailure.newNonRetryableFailureWithCause( String.format("Failed cleaning Gobblin job %s", jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY)), e.getClass().getName(), - e, - null + e ); } log.error("Failed to cleanup work dirs", e); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java index 973d078b6ff..9796868e4e1 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java @@ -17,6 +17,7 @@ package org.apache.gobblin.temporal.dynamic; +import java.net.URLEncoder; import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -65,6 +66,7 @@ public MalformedDirectiveException(String directive, String desc) { private static final String KEY_REGEX = "(\\w+(?:\\.\\w+)*)"; private static final String KEY_VALUE_REGEX = KEY_REGEX + "\\s*=\\s*(.*)"; + private static final String URL_ENCODING_CHARSET = "UTF-8"; private static final Pattern directivePattern = Pattern.compile(DIRECTIVE_REGEX); private static final Pattern keyPattern = Pattern.compile(KEY_REGEX); private static final Pattern keyValuePattern = Pattern.compile(KEY_VALUE_REGEX); @@ -118,15 +120,52 @@ public ScalingDirective parse(String directive) { } } + public static String asString(ScalingDirective directive) { + StringBuilder sb = new StringBuilder(); + sb.append(directive.getTimestampEpochMillis()).append('.').append(directive.getProfileName()).append('=').append(directive.getSetPoint()); + directive.getOptDerivedFrom().ifPresent(derivedFrom -> { + sb.append(',').append(derivedFrom.getBasisProfileName()); + sb.append(derivedFrom.getOverlay() instanceof ProfileOverlay.Adding ? "+(" : "-("); + ProfileOverlay overlay = derivedFrom.getOverlay(); + if (overlay instanceof ProfileOverlay.Adding) { + ProfileOverlay.Adding adding = (ProfileOverlay.Adding) overlay; + for (ProfileOverlay.KVPair kv : adding.getAdditionPairs()) { + sb.append(kv.getKey()).append('=').append(urlEncode(kv.getValue())).append(", "); + } + if (adding.getAdditionPairs().size() > 0) { + sb.setLength(sb.length() - 2); // remove trailing ", " + } + } else { + ProfileOverlay.Removing removing = (ProfileOverlay.Removing) overlay; + for (String key : removing.getRemovalKeys()) { + sb.append(key).append(", "); + } + if (removing.getRemovalKeys().size() > 0) { + sb.setLength(sb.length() - 2); // remove trailing ", " + } + } + sb.append(')'); + }); + return sb.toString(); + } + private static String identifyProfileName(String profileId) { return profileId.equals(BASELINE_ID) ? WorkforceProfiles.BASELINE_NAME : profileId; } private static String urlDecode(String directive, String s) { try { - return java.net.URLDecoder.decode(s, "UTF-8"); + return java.net.URLDecoder.decode(s, URL_ENCODING_CHARSET); } catch (java.io.UnsupportedEncodingException e) { throw new MalformedDirectiveException(directive, "unable to URL-decode - {{" + s + "}}"); } } + + private static String urlEncode(String s) { + try { + return URLEncoder.encode(s, URL_ENCODING_CHARSET); + } catch (java.io.UnsupportedEncodingException e) { + throw new RuntimeException("THIS SHOULD BE IMPOSSIBLE, given we used '" + URL_ENCODING_CHARSET + "' with {{" + s + "}}", e); + } + } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java index 574f4966411..5b2e0d4f123 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java @@ -144,21 +144,20 @@ public void parseRemovingOverlayWithBaselineProfile() { new ProfileOverlay.Removing(Lists.newArrayList("a.b", "c.d")))); } - @DataProvider(name = "funkyButValidDirectives") public String[][] validDirectives() { return new String[][]{ // null overlay upon unnamed baseline profile (null overlay functions as the 'identity element'): - {"1728435970.my_profile=24,+()"}, - {"1728435970.my_profile=24,-()"}, - {"1728435970.my_profile=24;+()"}, - {"1728435970.my_profile=24;-()"}, + { "1728435970.my_profile=24,+()" }, + { "1728435970.my_profile=24,-()" }, + { "1728435970.my_profile=24;+()" }, + { "1728435970.my_profile=24;-()" }, // null overlay upon named profile: - {"1728435970.my_profile=24,foo+()"}, - {"1728435970.my_profile=24,foo-()"}, - {"1728435970.my_profile=24;foo+()"}, - {"1728435970.my_profile=24;foo-()"}, + { "1728435970.my_profile=24,foo+()" }, + { "1728435970.my_profile=24,foo-()" }, + { "1728435970.my_profile=24;foo+()" }, + { "1728435970.my_profile=24;foo-()" }, // seemingly separator mismatch, but in fact the NOT-separator is part of the value (e.g. a="7;m=sixteen"): { "1728439210.new_profile=16,bar+(a=7;m=sixteen)" }, @@ -178,6 +177,27 @@ public void parseValidDirectives(String directive) { Assert.assertNotNull(parser.parse(directive)); } + @DataProvider(name = "validDirectivesToRoundTrip") + public String[][] validDirectivesForRoundTrip() { + return new String[][]{ + { "2.some_profile=15" }, + { "6.extra_profile=9,the_basis+(a.b=foo, c.d=bar)" }, + { "6.extra_profile=9,the_basis-(a.b, c.d)" }, + // funky ones: + { "1728435970.my_profile=24,+()" }, + { "1728435970.my_profile=24,-()" }, + { "1728435970.my_profile=24,foo+()" }, + { "1728435970.my_profile=24,foo-()" } + }; + } + + @Test( + expectedExceptions = {}, + dataProvider = "validDirectivesToRoundTrip" + ) + public void roundTripAsStringAfterParse(String directive) { + Assert.assertEquals(ScalingDirectiveParser.asString(parser.parse(directive)), directive); + } @DataProvider(name = "invalidDirectives") public String[][] invalidDirectives() { From 7ac5f74cd89e099d665f6315701e867432f39d3d Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Tue, 29 Oct 2024 13:38:07 -0700 Subject: [PATCH 04/11] fix checkstyle --- .../temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java index eea5d14ebe7..45e2ecb9b16 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/work/AbstractEagerFsDirBackedWorkload.java @@ -26,10 +26,8 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -import lombok.AccessLevel; import lombok.Getter; import lombok.NonNull; -import lombok.Setter; import lombok.extern.slf4j.Slf4j; import com.fasterxml.jackson.annotation.JsonIgnore; From cf6950b985cbb4b4d629daa9ab97a8b4611c0ce1 Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Tue, 29 Oct 2024 13:41:54 -0700 Subject: [PATCH 05/11] misc. impl naming improvements and reformatting --- .../temporal/dynamic/WorkforcePlan.java | 1 - .../temporal/dynamic/WorkforceStaffing.java | 10 +-- .../temporal/dynamic/WorkforcePlanTest.java | 76 +++++++++---------- 3 files changed, 43 insertions(+), 44 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java index 33c736b9f3b..76b5a44a497 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java @@ -121,7 +121,6 @@ public synchronized void reviseWhenNewer(List directives, Cons try { revise(directive); } catch (IllegalRevisionException ire) { - System.err.println("uh oh it's: " + ire); illegalRevisionHandler.accept(ire); } }); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java index b9f66acd970..48860bca819 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java @@ -36,7 +36,7 @@ public class WorkforceStaffing { @Data private static class SetPoint { - private final int point; + private final int numWorkers; private final long provenanceEpochMillis; // for debuggability } @@ -63,7 +63,7 @@ public static WorkforceStaffing initializeStaffing(int initialBaselineSetPoint, } public Optional getStaffing(String profileName) { - return Optional.ofNullable(setPointByName.get(profileName)).map(SetPoint::getPoint); + return Optional.ofNullable(setPointByName.get(profileName)).map(SetPoint::getNumWorkers); } public void reviseStaffing(String profileName, int setPoint, long provenanceEpochMillis) { @@ -83,13 +83,13 @@ public synchronized StaffingDeltas calcDeltas(WorkforceStaffing reference, Workf // not expecting any profile earlier in `reference` to no longer be set... (but defensive coding nonetheless) List profileDeltas = frozenReferenceSetPointsByName.entrySet().stream() .filter(entry -> !this.setPointByName.containsKey(entry.getKey())) - .map(entry -> new StaffingDeltas.ProfileDelta(profiles.getOrThrow(entry.getKey()), 0 - entry.getValue().getPoint(), UNKNOWN_PROVENANCE_EPOCH_MILLIS)) + .map(entry -> new StaffingDeltas.ProfileDelta(profiles.getOrThrow(entry.getKey()), 0 - entry.getValue().getNumWorkers(), UNKNOWN_PROVENANCE_EPOCH_MILLIS)) .collect(Collectors.toList()); profileDeltas.addAll(this.setPointByName.entrySet().stream().map(entry -> { - Optional optEquivReferenceSetPoint = Optional.ofNullable(frozenReferenceSetPointsByName.get(entry.getKey())).map(SetPoint::getPoint); + Optional optEquivReferenceSetPoint = Optional.ofNullable(frozenReferenceSetPointsByName.get(entry.getKey())).map(SetPoint::getNumWorkers); return new StaffingDeltas.ProfileDelta( profiles.getOrThrow(entry.getKey()), - entry.getValue().getPoint() - optEquivReferenceSetPoint.orElse(0), + entry.getValue().getNumWorkers() - optEquivReferenceSetPoint.orElse(0), entry.getValue().getProvenanceEpochMillis()); } ).filter(delta -> !delta.isUnchanged()) diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java index 8bc24897e9b..cdaa875f264 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java @@ -40,17 +40,10 @@ public void setUp() { plan = new WorkforcePlan(baselineConfig, initialBaselineSetPoint); } - private static ScalingDirective createNewProfileDirective(String profileName, int setPoint, long epochMillis, String basisProfileName) { - return new ScalingDirective(profileName, setPoint, epochMillis, Optional.of( - new ProfileDerivation(basisProfileName, new ProfileOverlay.Adding(Lists.newArrayList( - new ProfileOverlay.KVPair("key1", "new_value"), - new ProfileOverlay.KVPair("key4", "value4")))))); - } - @Test public void reviseWithValidReSetPoint() throws WorkforcePlan.IllegalRevisionException { - plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7,10000L)); - plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 1,20000L)); + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7, 10000L)); + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 1, 20000L)); Assert.assertEquals(plan.getLastRevisionEpochMillis(), 20000L); Assert.assertEquals(plan.getNumProfiles(), 1); } @@ -59,7 +52,7 @@ public void reviseWithValidReSetPoint() throws WorkforcePlan.IllegalRevisionExce public void reviseWithValidDerivation() throws WorkforcePlan.IllegalRevisionException { Assert.assertEquals(plan.getLastRevisionEpochMillis(), WorkforceStaffing.INITIALIZATION_PROVENANCE_EPOCH_MILLIS); Assert.assertEquals(plan.getNumProfiles(), 1); - ScalingDirective directive = createNewProfileDirective("new_profile", 5,10000L, WorkforceProfiles.BASELINE_NAME); + ScalingDirective directive = createNewProfileDirective("new_profile", 5, 10000L, WorkforceProfiles.BASELINE_NAME); plan.revise(directive); Assert.assertEquals(plan.getLastRevisionEpochMillis(), 10000L); @@ -74,14 +67,14 @@ public void reviseWhenNewerIgnoresOutOfOrderDirectives() throws WorkforcePlan.Il Assert.assertEquals(plan.getLastRevisionEpochMillis(), WorkforceStaffing.INITIALIZATION_PROVENANCE_EPOCH_MILLIS); Assert.assertEquals(plan.getNumProfiles(), 1); plan.reviseWhenNewer(Lists.newArrayList( - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 2,100L), - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 3,500L), - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 4,200L), - createNewProfileDirective("new_profile", 5,400L, WorkforceProfiles.BASELINE_NAME), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 2, 100L), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 3, 500L), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 4, 200L), + createNewProfileDirective("new_profile", 5, 400L, WorkforceProfiles.BASELINE_NAME), // NOTE: the second attempt at derivation is NOT judged a duplicate, as the outdated timestamp of first attempt (above) meant it was ignored! - createNewProfileDirective("new_profile", 6,600L, WorkforceProfiles.BASELINE_NAME), - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7,800L), - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8,700L) + createNewProfileDirective("new_profile", 6, 600L, WorkforceProfiles.BASELINE_NAME), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7, 800L), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8, 700L) ), failure -> numErrors.incrementAndGet()); Assert.assertEquals(plan.getLastRevisionEpochMillis(), 800L); @@ -95,19 +88,19 @@ public void reviseWhenNewerIgnoresOutOfOrderDirectives() throws WorkforcePlan.Il public void reviseWhenNewerSwallowsErrors() throws WorkforcePlan.IllegalRevisionException { AtomicInteger numErrors = new AtomicInteger(0); plan.reviseWhenNewer(Lists.newArrayList( - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 1,100L), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 1, 100L), // (1) error: `UnrecognizedProfile` - new ScalingDirective("unknown_profile", 2,250L), - createNewProfileDirective("new_profile", 3,200L, WorkforceProfiles.BASELINE_NAME), + new ScalingDirective("unknown_profile", 2, 250L), + createNewProfileDirective("new_profile", 3, 200L, WorkforceProfiles.BASELINE_NAME), // (2) error: `Redefinition` - createNewProfileDirective("new_profile", 4,450L, WorkforceProfiles.BASELINE_NAME), - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 5,300L), + createNewProfileDirective("new_profile", 4, 450L, WorkforceProfiles.BASELINE_NAME), + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 5, 300L), // (3) error: `UnknownBasis` - createNewProfileDirective("other_profile", 6,550L, "never_defined"), - new ScalingDirective("new_profile", 7,400L), + createNewProfileDirective("other_profile", 6, 550L, "never_defined"), + new ScalingDirective("new_profile", 7, 400L), // ignored: out-of-order timestamp (not an error... see: `reviseWhenNewerIgnoresOutOfOrderDirectives`) - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8,350L), - createNewProfileDirective("another", 9,500L, "new_profile") + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8, 350L), + createNewProfileDirective("another", 9, 500L, "new_profile") ), failure -> numErrors.incrementAndGet()); Assert.assertEquals(plan.getLastRevisionEpochMillis(), 500L); @@ -120,12 +113,12 @@ public void reviseWhenNewerSwallowsErrors() throws WorkforcePlan.IllegalRevision @Test public void calcStaffingDeltas() throws WorkforcePlan.IllegalRevisionException { - plan.revise(createNewProfileDirective("new_profile", 3,10L, WorkforceProfiles.BASELINE_NAME)); - plan.revise(createNewProfileDirective("other_profile", 8,20L, "new_profile")); - plan.revise(createNewProfileDirective("another", 7,30L, "new_profile")); - plan.revise(new ScalingDirective("new_profile", 5,40L)); - plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 6,50L)); - plan.revise(new ScalingDirective("another", 4,60L)); + plan.revise(createNewProfileDirective("new_profile", 3, 10L, WorkforceProfiles.BASELINE_NAME)); + plan.revise(createNewProfileDirective("other_profile", 8, 20L, "new_profile")); + plan.revise(createNewProfileDirective("another", 7, 30L, "new_profile")); + plan.revise(new ScalingDirective("new_profile", 5, 40L)); + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 6, 50L)); + plan.revise(new ScalingDirective("another", 4, 60L)); Assert.assertEquals(plan.getLastRevisionEpochMillis(), 60L); Assert.assertEquals(plan.getNumProfiles(), 4); @@ -165,23 +158,30 @@ public void calcStaffingDeltas() throws WorkforcePlan.IllegalRevisionException { @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.OutdatedDirective.class) public void reviseWithOutdatedDirective() throws WorkforcePlan.IllegalRevisionException { - plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7,30000L)); - plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 12,8000L)); + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7, 30000L)); + plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 12, 8000L)); } @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.UnrecognizedProfile.class) public void reviseWithUnrecognizedProfileDirective() throws WorkforcePlan.IllegalRevisionException { - plan.revise(new ScalingDirective("unknown_profile", 7,10000L)); + plan.revise(new ScalingDirective("unknown_profile", 7, 10000L)); } @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.Redefinition.class) public void reviseWithRedefinitionDirective() throws WorkforcePlan.IllegalRevisionException { - plan.revise(createNewProfileDirective("new_profile", 5,10000L, WorkforceProfiles.BASELINE_NAME)); - plan.revise(createNewProfileDirective("new_profile", 9,20000L, WorkforceProfiles.BASELINE_NAME)); + plan.revise(createNewProfileDirective("new_profile", 5, 10000L, WorkforceProfiles.BASELINE_NAME)); + plan.revise(createNewProfileDirective("new_profile", 9, 20000L, WorkforceProfiles.BASELINE_NAME)); } @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.UnknownBasis.class) public void reviseWithUnknownBasisDirective() throws WorkforcePlan.IllegalRevisionException { - plan.revise(createNewProfileDirective("new_profile", 5,10000L, "never_defined")); + plan.revise(createNewProfileDirective("new_profile", 5, 10000L, "never_defined")); + } + + private static ScalingDirective createNewProfileDirective(String profileName, int setPoint, long epochMillis, String basisProfileName) { + return new ScalingDirective(profileName, setPoint, epochMillis, Optional.of( + new ProfileDerivation(basisProfileName, new ProfileOverlay.Adding(Lists.newArrayList( + new ProfileOverlay.KVPair("key1", "new_value"), + new ProfileOverlay.KVPair("key4", "value4")))))); } } From b27cb67403eddbe9480358cf760733ad93c48df2 Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Tue, 29 Oct 2024 23:29:31 -0700 Subject: [PATCH 06/11] Add (HDFS) `FsScalingDirectiveSource` with support for overlay placeholder defs --- .../dynamic/FsScalingDirectiveSource.java | 127 +++++++++ .../dynamic/ScalingDirectiveParser.java | 96 +++++-- .../dynamic/ScalingDirectiveSource.java | 27 ++ .../dynamic/FsScalingDirectiveSourceTest.java | 260 ++++++++++++++++++ .../dynamic/ScalingDirectiveParserTest.java | 165 +++++++++-- 5 files changed, 641 insertions(+), 34 deletions(-) create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveSource.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSourceTest.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java new file mode 100644 index 00000000000..395aedadaf7 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynamic; + +import com.google.common.base.Charsets; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +@Slf4j +public class FsScalingDirectiveSource implements ScalingDirectiveSource { + private final FileSystem fileSystem; + private final Path dirPath; + private final Optional optErrorsPath; + private final ScalingDirectiveParser parser = new ScalingDirectiveParser(); + + public FsScalingDirectiveSource(FileSystem fileSystem, String directivesDirPath, Optional optErrorDirPath) { + this.fileSystem = fileSystem; + this.dirPath = new Path(directivesDirPath); + this.optErrorsPath = optErrorDirPath.map(Path::new); + } + + // TODO: describe purpose of constraint (to preclude late insertion/edits of the directives stream) - + // verify and only return directives whose stated (in filename) timestamp order matches `FileStatus` modtime order + @Override + public List getScalingDirectives() throws IOException { + List> directiveWithFileStatus = new ArrayList<>(); + for (FileStatus fileStatus : fileSystem.listStatus(dirPath)) { + if (!fileStatus.isFile()) { + log.warn("Ignoring non-file object: " + fileStatus); + optAcknowledgeError(fileStatus, "non-file (not an actual)"); + } else { + String fileName = fileStatus.getPath().getName(); + try { + try { + directiveWithFileStatus.add(new ImmutablePair<>(parseScalingDirective(fileName), fileStatus)); + } catch (ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition needsDefinition) { + // directive used placeholder syntax to indicate the overlay definition resides inside its file... so open the file to load that def + log.info("Loading overlay definition for directive {{" + fileName + "}} from: " + fileStatus); + String overlayDef = slurpFileAsString(fileStatus.getPath()); + directiveWithFileStatus.add(new ImmutablePair<>(needsDefinition.retryParsingWithDefinition(overlayDef), fileStatus)); + } + } catch (ScalingDirectiveParser.InvalidSyntaxException e) { + log.warn("Ignoring unparseable scaling directive {{" + fileName + "}}: " + fileStatus + " - " + e.getClass().getName() + ": " + e.getMessage()); + optAcknowledgeError(fileStatus, "unparseable"); + } + } + } + + // verify and only return directives whose ordering of stated (in filename) timestamp matches `FileStatus` modtime order + List directives = new ArrayList<>(); + // NOTE: for deterministic total-ordering, sort by path, rather than by timestamp, in case of modtime tie (given only secs granularity) + directiveWithFileStatus.sort(Comparator.comparing(p -> p.getValue().getPath())); + long latestValidModTime = -1; + for (Map.Entry entry : directiveWithFileStatus) { + long thisModTime = entry.getValue().getModificationTime(); + if (thisModTime < latestValidModTime) { // do NOT reject equal (non-increasing) modtime, given granularity of epoch seconds + log.warn("Ignoring out-of-order scaling directive " + entry.getKey() + " since FS modTime " + thisModTime + " precedes last observed " + + latestValidModTime + ": " + entry.getValue()); + optAcknowledgeError(entry.getValue(), "out-of-order"); + } else { + directives.add(entry.getKey()); + latestValidModTime = thisModTime; + } + } + return directives; + } + + // ack error by moving the bad/non-directive to a separate errors dir + protected void optAcknowledgeError(FileStatus fileStatus, String desc) { + this.optErrorsPath.ifPresent(errorsPath -> + moveToErrors(fileStatus, errorsPath, desc) + ); + } + + // move broken/ignored directives into a separate directory, as an observability-enhancing ack of its rejection + protected void moveToErrors(FileStatus badDirectiveStatus, Path errorsPath, String desc) { + Path badDirectivePath = badDirectiveStatus.getPath(); + try { + if (!this.fileSystem.rename(badDirectivePath, new Path(errorsPath, badDirectivePath.getName()))) { + throw new RuntimeException(); // unclear how to obtain more info about such a failure + } + } catch (IOException e) { + log.warn("Failed to move " + desc + " directive {{" + badDirectiveStatus.getPath() + "}} to '" + errorsPath + "'... leaving in place", e); + } catch (RuntimeException e) { + log.warn("Failed to move " + desc + " directive {{" + badDirectiveStatus.getPath() + "}} to '" + errorsPath + "' [unknown reason]... leaving in place"); + } + } + + private ScalingDirective parseScalingDirective(String fileName) + throws ScalingDirectiveParser.InvalidSyntaxException, ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition { + return parser.parse(fileName); + } + + protected String slurpFileAsString(Path path) throws IOException { + try (InputStream is = this.fileSystem.open(path)) { + return IOUtils.toString(is, Charsets.UTF_8); + } + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java index 9796868e4e1..8719692203d 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java @@ -19,10 +19,12 @@ import java.net.URLEncoder; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; @@ -53,47 +55,104 @@ */ @Slf4j public class ScalingDirectiveParser { - public static class MalformedDirectiveException extends IllegalArgumentException { + public static class InvalidSyntaxException extends Exception { private final String directive; - public MalformedDirectiveException(String directive, String desc) { + public InvalidSyntaxException(String directive, String desc) { super("error: " + desc + ", in ==>" + directive + "<=="); this.directive = directive; } } - private static final String DIRECTIVE_REGEX = "(?x) \\s* (\\d+) \\s* \\. \\s* (\\w* | baseline\\(\\)) \\s* = \\s* (\\d+) " - + "(?: \\s* ([;,]) \\s* (\\w* | baseline\\(\\)) \\s* (?: (\\+ \\s* \\( \\s* ([^)]*?) \\s* \\) ) | (- \\s* \\( \\s* ([^)]*?) \\s* \\) ) ) )? \\s*"; + public static class OverlayPlaceholderNeedsDefinition extends RuntimeException { + private final String directive; + private final String overlaySep; + private final boolean isAdding; + // ATTENTION: explicitly managed, rather than making this a non-static inner class so `definePlaceholder` may be `static` for testing, while avoiding: + // Static declarations in inner classes are not supported at language level '8' + private final ScalingDirectiveParser parser; + + private OverlayPlaceholderNeedsDefinition(String directive, String overlaySep, boolean isAdding, ScalingDirectiveParser enclosing) { + super("overlay placeholder, in ==>" + directive + "<=="); + this.directive = directive; + this.overlaySep = overlaySep; + this.isAdding = isAdding; + this.parser = enclosing; + } + + // doesn't allow recursive placeholding... + public ScalingDirective retryParsingWithDefinition(String overlayDefinition) throws InvalidSyntaxException { + try { + return this.parser.parse(definePlaceholder(this.directive, this.overlaySep, this.isAdding, overlayDefinition)); + } catch (OverlayPlaceholderNeedsDefinition e) { + throw new InvalidSyntaxException(this.directive, "overlay placeholder definition must not be itself another placeholder"); + } + } + + protected static String definePlaceholder(String directiveWithPlaceholder, String overlaySep, boolean isAdding, String overlayDefinition) { + // use care to selectively `urlEncode` parts (but NOT the entire string), to avoid disrupting syntactic chars, like [,;=] + String urlEncodedOverlayDef = Arrays.stream(overlayDefinition.split("\\s*" + overlaySep + "\\s*", -1)) // (neg. limit to disallow trailing empty strings) + .map(kvPair -> { + String[] kv = kvPair.split("\\s*=\\s*", 2); + if (isAdding && kv.length > 1) { + return kv[0] + '=' + urlEncode(kv[1]); + } else { + return kvPair; + } + }).collect(Collectors.joining(overlaySep)); + + // correct any double-encoding of '%', in case it arrived url-encoded + return directiveWithPlaceholder.replace(OVERLAY_DEFINITION_PLACEHOLDER, urlEncodedOverlayDef.replace("%25", "%")); + } + } + + + // TODO: also support non-inline overlay definitions - "(...)" + // consider an additional trailing "|" (or "," / ";") syntax when the additional props are only needed post-launch + // since we're primarily designed for HDFS file names, in addition, a max identifier length (to fit within HDFS path segment limit == 255) + // org.apache.hadoop.hdfs.protocol.FSLimitException$PathComponentTooLongException: \ + // The maximum path component name limit of ... in directory ... is exceeded: limit=255 length=256 + // 200 = 255 [limit] - 16 [digit timestamp] - 1 ['.'] - 1 ['='] - 1 [',' / ';'] - 6 ['+(...)' / '-(...)'] - 30 [reserved... for future] + // current millis-precision epoch timestamp requires 10 chars, but we reserve 16 for future-proofing to nanos-precision + // hence, neither (of the max two) profile identifiers may exceed 100 chars. + // TODO: syntax to indicate removing an attr during an addition + private static final String DIRECTIVE_REGEX = "(?x) (?s) ^ \\s* (\\d+) \\s* \\. \\s* (\\w* | baseline\\(\\)) \\s* = \\s* (\\d+) " + + "(?: \\s* ([;,]) \\s* (\\w* | baseline\\(\\)) \\s* (?: (\\+ \\s* \\( \\s* ([^)]*?) \\s* \\) ) | (- \\s* \\( \\s* ([^)]*?) \\s* \\) ) ) )? \\s* $"; + + public static final int MAX_PROFILE_IDENTIFIER_LENGTH = 100; + public static final String URL_ENCODING_CHARSET = "UTF-8"; + public static final String OVERLAY_DEFINITION_PLACEHOLDER = "..."; private static final String KEY_REGEX = "(\\w+(?:\\.\\w+)*)"; private static final String KEY_VALUE_REGEX = KEY_REGEX + "\\s*=\\s*(.*)"; - private static final String URL_ENCODING_CHARSET = "UTF-8"; private static final Pattern directivePattern = Pattern.compile(DIRECTIVE_REGEX); private static final Pattern keyPattern = Pattern.compile(KEY_REGEX); private static final Pattern keyValuePattern = Pattern.compile(KEY_VALUE_REGEX); private static final String BASELINE_ID = "baseline()"; - public ScalingDirective parse(String directive) { + public ScalingDirective parse(String directive) throws InvalidSyntaxException { Matcher parsed = directivePattern.matcher(directive); if (parsed.matches()) { long timestamp = Long.parseLong(parsed.group(1)); String profileId = parsed.group(2); - String profileName = identifyProfileName(profileId); + String profileName = identifyProfileName(profileId, directive); int setpoint = Integer.parseInt(parsed.group(3)); Optional optDerivedFrom = Optional.empty(); String overlayIntroSep = parsed.group(4); if (overlayIntroSep != null) { - String basisProfileName = identifyProfileName(parsed.group(5)); + String basisProfileName = identifyProfileName(parsed.group(5), directive); if (parsed.group(6) != null) { // '+' == adding List additions = new ArrayList<>(); String additionsStr = parsed.group(7); - if (!additionsStr.equals("")) { + if (additionsStr.equals(OVERLAY_DEFINITION_PLACEHOLDER)) { + throw new OverlayPlaceholderNeedsDefinition(directive, overlayIntroSep, true, this); + } else if (!additionsStr.equals("")) { for (String addStr : additionsStr.split("\\s*" + overlayIntroSep + "\\s*", -1)) { // (negative limit to disallow trailing empty strings) Matcher keyValueParsed = keyValuePattern.matcher(addStr); if (keyValueParsed.matches()) { additions.add(new ProfileOverlay.KVPair(keyValueParsed.group(1), urlDecode(directive, keyValueParsed.group(2)))); } else { - throw new MalformedDirectiveException(directive, "unable to parse key-value pair - {{" + addStr + "}}"); + throw new InvalidSyntaxException(directive, "unable to parse key-value pair - {{" + addStr + "}}"); } } } @@ -101,13 +160,15 @@ public ScalingDirective parse(String directive) { } else { // '-' == removing List removalKeys = new ArrayList<>(); String removalsStr = parsed.group(9); - if (!removalsStr.equals("")) { + if (removalsStr.equals(OVERLAY_DEFINITION_PLACEHOLDER)) { + throw new OverlayPlaceholderNeedsDefinition(directive, overlayIntroSep, false, this); + } else if (!removalsStr.equals("")) { for (String removeStr : removalsStr.split("\\s*" + overlayIntroSep + "\\s*", -1)) { // (negative limit to disallow trailing empty strings) Matcher keyParsed = keyPattern.matcher(removeStr); if (keyParsed.matches()) { removalKeys.add(keyParsed.group(1)); } else { - throw new MalformedDirectiveException(directive, "unable to parse key - {{" + removeStr + "}}"); + throw new InvalidSyntaxException(directive, "unable to parse key - {{" + removeStr + "}}"); } } } @@ -116,7 +177,7 @@ public ScalingDirective parse(String directive) { } return new ScalingDirective(profileName, setpoint, timestamp, optDerivedFrom); } else { - throw new MalformedDirectiveException(directive, "invalid syntax"); + throw new InvalidSyntaxException(directive, "invalid syntax"); } } @@ -149,15 +210,18 @@ public static String asString(ScalingDirective directive) { return sb.toString(); } - private static String identifyProfileName(String profileId) { + private static String identifyProfileName(String profileId, String directive) throws InvalidSyntaxException { + if (profileId.length() > MAX_PROFILE_IDENTIFIER_LENGTH) { + throw new InvalidSyntaxException(directive, "profile ID exceeds length limit (of " + MAX_PROFILE_IDENTIFIER_LENGTH + "): '" + profileId + "'"); + } return profileId.equals(BASELINE_ID) ? WorkforceProfiles.BASELINE_NAME : profileId; } - private static String urlDecode(String directive, String s) { + private static String urlDecode(String directive, String s) throws InvalidSyntaxException { try { return java.net.URLDecoder.decode(s, URL_ENCODING_CHARSET); } catch (java.io.UnsupportedEncodingException e) { - throw new MalformedDirectiveException(directive, "unable to URL-decode - {{" + s + "}}"); + throw new InvalidSyntaxException(directive, "unable to URL-decode - {{" + s + "}}"); } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveSource.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveSource.java new file mode 100644 index 00000000000..98f1632e6f1 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveSource.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynamic; + +import java.io.IOException; +import java.util.List; + + +public interface ScalingDirectiveSource extends Cloneable { + // TODO - document! (impl may choose to give only newer directives, not previously returned... or to return them all) + List getScalingDirectives() throws IOException; +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSourceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSourceTest.java new file mode 100644 index 00000000000..fad35c94e5e --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSourceTest.java @@ -0,0 +1,260 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.dynamic; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import com.google.common.collect.Streams; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import org.testng.collections.Lists; + +import org.apache.gobblin.util.io.SeekableFSInputStream; + + +public class FsScalingDirectiveSourceTest { + + private static final String DIRECTIVES_DIR = "/test/dynamic/directives"; + private static final String ERRORS_DIR = "/test/dynamic/errors"; + private FileSystem fileSystem; + private FsScalingDirectiveSource source; + private static final ScalingDirectiveParser parser = new ScalingDirectiveParser(); + + @BeforeMethod + public void setUp() { + fileSystem = Mockito.mock(FileSystem.class); + source = new FsScalingDirectiveSource(fileSystem, DIRECTIVES_DIR, Optional.of(ERRORS_DIR)); + } + + @Test + public void getScalingDirectivesWhenAllValidFiles() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + String[] fileNames = { + "1700010000.=4", + "1700020000.new_profile=7,+(a.b.c=7,x.y=five)", + "1700030000.another_profile=3,+(a.b.c=8,x.y=six)", + "1700040000.new_profile=2" + }; + FileStatus[] fileStatuses = Streams.mapWithIndex(Arrays.stream(fileNames), (fileName, i) -> + createFileStatus(fileName, 1000 * (i + 1)) + ).toArray(FileStatus[]::new); + Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); + List directives = source.getScalingDirectives(); + + Assert.assertEquals(directives.size(), 4); + for (int i = 0; i < directives.size(); i++) { + Assert.assertEquals(directives.get(i), parseDirective(fileNames[i]), "fileNames[" + i + "] = " + fileNames[i]); + } + } + + @Test + public void getScalingDirectivesWhileIgnoringInvalidEntries() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + String[] fileNames = { + "1700010000.=4", + // still returned... although it would later be rejected as `WorkforcePlan.IllegalRevisionException.UnrecognizedProfile` + "1700020000.new_profile=2", + "1700030000.new_profile=7,+(a.b.c=7,x.y=five)", + // rejected: illegal syntax will fail to parse + "completely invalid", + "1700040000.another_profile=3,+(a.b.c=8,x.y=six)", + // rejected: because we later mock this as a dir, but a directive MUST be a file + "1700046000.acutally_a_dir=6,-(b.a,y.x)", + "1700050000.new_profile=9", + // rejected: because Removing must list key names, NOT key-value pairs + "1700055000.bad_directive=69,my_profile-(x=y,z=1)" + }; + FileStatus[] fileStatuses = Streams.mapWithIndex(Arrays.stream(fileNames), (fileName, i) -> { + boolean isFile = !fileName.contains("_a_dir="); + return createFileStatus(fileName, 1000 * (i + 1), isFile); + }).toArray(FileStatus[]::new); + Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); + List directives = source.getScalingDirectives(); + + Assert.assertEquals(directives.size(), 5); + Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); + Assert.assertEquals(directives.get(1), parseDirective(fileNames[1]), "fileNames[" + 1 + "] = " + fileNames[1]); + Assert.assertEquals(directives.get(2), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); + Assert.assertEquals(directives.get(3), parseDirective(fileNames[4]), "fileNames[" + 4 + "] = " + fileNames[4]); + Assert.assertEquals(directives.get(4), parseDirective(fileNames[6]), "fileNames[" + 6 + "] = " + fileNames[6]); + + // lastly, verify `ERRORS_DIR` acknowledgements (i.e. FS object rename) work as expected: + ArgumentCaptor sourcePathCaptor = ArgumentCaptor.forClass(Path.class); + ArgumentCaptor destPathCaptor = ArgumentCaptor.forClass(Path.class); + Mockito.verify(fileSystem, Mockito.times(fileNames.length - directives.size())) + .rename(sourcePathCaptor.capture(), destPathCaptor.capture()); + + List expectedErrorFileNames = Lists.newArrayList(fileNames[3], fileNames[5], fileNames[7]); + List expectedErrorDirectivePaths = expectedErrorFileNames.stream() + .map(fileName -> new Path(DIRECTIVES_DIR, fileName)) + .collect(Collectors.toList()); + List expectedErrorPostRenamePaths = expectedErrorFileNames.stream() + .map(fileName -> new Path(ERRORS_DIR, fileName)) + .collect(Collectors.toList()); + + Assert.assertEquals(sourcePathCaptor.getAllValues(), expectedErrorDirectivePaths); + Assert.assertEquals(destPathCaptor.getAllValues(), expectedErrorPostRenamePaths); + } + + @Test + public void getScalingDirectivesWhileIgnoringOutOfOrderEntries() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + String[] fileNames = { + "1700010000.=4", + "1700030000.new_profile=7,+(a.b.c=7,x.y=five)", + "1700040000.another_profile=3,+(a.b.c=8,x.y=six)", + "1700050000.new_profile=9" + }; + FileStatus[] fileStatuses = Streams.mapWithIndex(Arrays.stream(fileNames), (fileName, i) -> + // NOTE: elements [1] and [3] modtime will be 0, making them out of order against their directive timestamp (in their filename, like `1700030000.`) + createFileStatus(fileName, 1000 * (i + 1) * ((i + 1) % 2)) + ).toArray(FileStatus[]::new); + Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); + List directives = source.getScalingDirectives(); + + Assert.assertEquals(directives.size(), 2); + Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); + Assert.assertEquals(directives.get(1), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); + + // lastly, verify `ERRORS_DIR` acknowledgements (i.e. FS object rename) work as expected: + ArgumentCaptor sourcePathCaptor = ArgumentCaptor.forClass(Path.class); + ArgumentCaptor destPathCaptor = ArgumentCaptor.forClass(Path.class); + Mockito.verify(fileSystem, Mockito.times(fileNames.length - directives.size())) + .rename(sourcePathCaptor.capture(), destPathCaptor.capture()); + + List expectedErrorFileNames = Lists.newArrayList(fileNames[1], fileNames[3]); + List expectedErrorDirectivePaths = expectedErrorFileNames.stream() + .map(fileName -> new Path(DIRECTIVES_DIR, fileName)) + .collect(Collectors.toList()); + List expectedErrorPostRenamePaths = expectedErrorFileNames.stream() + .map(fileName -> new Path(ERRORS_DIR, fileName)) + .collect(Collectors.toList()); + + Assert.assertEquals(sourcePathCaptor.getAllValues(), expectedErrorDirectivePaths); + Assert.assertEquals(destPathCaptor.getAllValues(), expectedErrorPostRenamePaths); + } + + @Test + public void getScalingDirectivesWithOverlayPlaceholders() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + String[] fileNames = { + "1700010000.=4", + "1700020000.some_profile=9,+(...)", + "1700030000.other_profile=2,-(...)", + "1700040000.some_profile=3", + "1700050000.other_profile=10" + }; + String addingOverlayDef = "a.b.c=7,x.y=five"; // for [1] + String removingOverlayDef = "b.c,y.z.a"; // for [2] + FileStatus[] fileStatuses = Streams.mapWithIndex(Arrays.stream(fileNames), (fileName, i) -> + createFileStatus(fileName, 1000 * (i + 1)) + ).toArray(FileStatus[]::new); + Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); + Mockito.when(fileSystem.open(new Path(DIRECTIVES_DIR, fileNames[1]))).thenReturn(createInputStreamFromString(addingOverlayDef)); + Mockito.when(fileSystem.open(new Path(DIRECTIVES_DIR, fileNames[2]))).thenReturn(createInputStreamFromString(removingOverlayDef)); + List directives = source.getScalingDirectives(); + + Assert.assertEquals(directives.size(), fileNames.length); + Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); + Assert.assertEquals(directives.get(1), parseDirective(fileNames[1].replace("...", addingOverlayDef)), "fileNames[" + 1 + "] = " + fileNames[1]); + Assert.assertEquals(directives.get(2), parseDirective(fileNames[2].replace("...", removingOverlayDef)), "fileNames[" + 2 + "] = " + fileNames[2]); + Assert.assertEquals(directives.get(3), parseDirective(fileNames[3]), "fileNames[" + 3 + "] = " + fileNames[3]); + Assert.assertEquals(directives.get(4), parseDirective(fileNames[4]), "fileNames[" + 4 + "] = " + fileNames[4]); + + Mockito.verify(fileSystem, Mockito.never()).rename(Mockito.any(), Mockito.any()); + } + + @Test + public void getScalingDirectivesWithOverlayPlaceholdersButInvalidDefinitions() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + String[] fileNames = { + "1700020000.some_profile=9,+(...)", + "1700030000.other_profile=2,-(...)", + "1700070000.=10" + }; + // NOTE: switch these, so the overlay defs are invalid from `addingOverlayDef` with Removing and `removingOverlayDef` with Adding + String addingOverlayDef = "a.b.c=7,x.y=five"; // for [1] + String removingOverlayDef = "b.c,y.z.a"; // for [0] + FileStatus[] fileStatuses = Streams.mapWithIndex(Arrays.stream(fileNames), (fileName, i) -> + createFileStatus(fileName, 1000 * (i + 1)) + ).toArray(FileStatus[]::new); + Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); + Mockito.when(fileSystem.open(new Path(DIRECTIVES_DIR, fileNames[0]))).thenReturn(createInputStreamFromString(removingOverlayDef)); + Mockito.when(fileSystem.open(new Path(DIRECTIVES_DIR, fileNames[1]))).thenReturn(createInputStreamFromString(addingOverlayDef)); + List directives = source.getScalingDirectives(); + + Assert.assertEquals(directives.size(), 1); + Assert.assertEquals(directives.get(0), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); + + // lastly, verify `ERRORS_DIR` acknowledgements (i.e. FS object rename) work as expected: + ArgumentCaptor sourcePathCaptor = ArgumentCaptor.forClass(Path.class); + ArgumentCaptor destPathCaptor = ArgumentCaptor.forClass(Path.class); + Mockito.verify(fileSystem, Mockito.times(fileNames.length - directives.size())) + .rename(sourcePathCaptor.capture(), destPathCaptor.capture()); + + List expectedErrorFileNames = Lists.newArrayList(fileNames[0], fileNames[1]); + List expectedErrorDirectivePaths = expectedErrorFileNames.stream() + .map(fileName -> new Path(DIRECTIVES_DIR, fileName)) + .collect(Collectors.toList()); + List expectedErrorPostRenamePaths = expectedErrorFileNames.stream() + .map(fileName -> new Path(ERRORS_DIR, fileName)) + .collect(Collectors.toList()); + + Assert.assertEquals(sourcePathCaptor.getAllValues(), expectedErrorDirectivePaths); + Assert.assertEquals(destPathCaptor.getAllValues(), expectedErrorPostRenamePaths); + } + + @Test + public void getScalingDirectivesWithNoFiles() throws IOException { + FileStatus[] fileStatuses = {}; + Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); + Assert.assertTrue(source.getScalingDirectives().isEmpty()); + } + + @Test(expectedExceptions = IOException.class) + public void getScalingDirectivesWithIOException() throws IOException { + Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenThrow(new IOException()); + source.getScalingDirectives(); + } + + protected static ScalingDirective parseDirective(String s) throws ScalingDirectiveParser.InvalidSyntaxException { + return parser.parse(s); + } + + protected static FileStatus createFileStatus(String fileName, long modTime) { + return createFileStatus(fileName, modTime, true); + } + + protected static FileStatus createFileStatus(String fileName, long modTime, boolean isFile) { + return new FileStatus(0, !isFile, 0, 0, modTime, new Path(DIRECTIVES_DIR, fileName)); + } + + public static FSDataInputStream createInputStreamFromString(String input) { + return new FSDataInputStream(new SeekableFSInputStream(new ByteArrayInputStream(input.getBytes(StandardCharsets.UTF_8)))); + } +} \ No newline at end of file diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java index 5b2e0d4f123..cd0a08b7bf7 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java @@ -17,7 +17,11 @@ package org.apache.gobblin.temporal.dynamic; +import java.util.Arrays; import java.util.Optional; +import java.util.function.BiFunction; +import java.util.stream.IntStream; + import com.google.common.collect.Lists; import org.testng.annotations.DataProvider; @@ -30,7 +34,7 @@ public class ScalingDirectiveParserTest { private final ScalingDirectiveParser parser = new ScalingDirectiveParser(); @Test - public void parseSimpleDirective() { + public void parseSimpleDirective() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728435970.my_profile=24"); Assert.assertEquals(sd.getTimestampEpochMillis(), 1728435970L); Assert.assertEquals(sd.getProfileName(), "my_profile"); @@ -39,7 +43,7 @@ public void parseSimpleDirective() { } @Test - public void parseUnnamedBaselineProfile() { + public void parseUnnamedBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728436821.=12"); Assert.assertEquals(sd.getTimestampEpochMillis(), 1728436821L); Assert.assertEquals(sd.getProfileName(), WorkforceProfiles.BASELINE_NAME); @@ -48,13 +52,13 @@ public void parseUnnamedBaselineProfile() { } @Test - public void parseBaselineProfile() { + public void parseBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728436828.baseline()=6"); Assert.assertEquals(sd, new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 6, 1728436828L, Optional.empty())); } @Test - public void parseAddingOverlayWithCommaSep() { + public void parseAddingOverlayWithCommaSep() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728439210.new_profile=16,bar+(a.b.c=7,l.m=sixteen)"); Assert.assertEquals(sd.getTimestampEpochMillis(), 1728439210L); Assert.assertEquals(sd.getProfileName(), "new_profile"); @@ -67,21 +71,21 @@ public void parseAddingOverlayWithCommaSep() { } @Test - public void parseAddingOverlayWithSemicolonSep() { + public void parseAddingOverlayWithSemicolonSep() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728439223.new_profile=32;baz+( a.b.c=7 ; l.m.n.o=sixteen )"); Assert.assertEquals(sd, new ScalingDirective("new_profile", 32, 1728439223L, "baz", new ProfileOverlay.Adding( Lists.newArrayList(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m.n.o", "sixteen"))))); } @Test - public void parseAddingOverlayWithCommaSepUrlEncoded() { + public void parseAddingOverlayWithCommaSepUrlEncoded() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728460832.new_profile=16,baa+(a.b.c=7,l.m=sixteen%2C%20again)"); Assert.assertEquals(sd, new ScalingDirective("new_profile", 16, 1728460832L, "baa", new ProfileOverlay.Adding( Lists.newArrayList(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m", "sixteen, again"))))); } @Test - public void parseRemovingOverlayWithCommaSep() { + public void parseRemovingOverlayWithCommaSep() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728436436.other_profile=9,my_profile-( x , y.z )"); Assert.assertEquals(sd.getTimestampEpochMillis(), 1728436436L); Assert.assertEquals(sd.getProfileName(), "other_profile"); @@ -93,14 +97,14 @@ public void parseRemovingOverlayWithCommaSep() { } @Test - public void parseRemovingOverlayWithSemicolonSep() { + public void parseRemovingOverlayWithSemicolonSep() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728436499.other_profile=9;my_profile-(x.y;z.z)"); Assert.assertEquals(sd, new ScalingDirective("other_profile", 9, 1728436499L, "my_profile", new ProfileOverlay.Removing(Lists.newArrayList("x.y", "z.z")))); } @Test - public void parseAddingOverlayWithWhitespace() { + public void parseAddingOverlayWithWhitespace() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse(" 1728998877 . another = 999 ; wow + ( t.r = jump%20 ; cb.az = foo%20#%20111 ) "); Assert.assertEquals(sd, new ScalingDirective("another", 999, 1728998877L, "wow", new ProfileOverlay.Adding( Lists.newArrayList(new ProfileOverlay.KVPair("t.r", "jump "), @@ -108,14 +112,14 @@ public void parseAddingOverlayWithWhitespace() { } @Test - public void parseRemovingOverlayWithWhitespace() { + public void parseRemovingOverlayWithWhitespace() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse(" 1728334455 . also = 77 , really - ( t.r , cb.az ) "); Assert.assertEquals(sd, new ScalingDirective("also", 77, 1728334455L, "really", new ProfileOverlay.Removing(Lists.newArrayList("t.r", "cb.az")))); } @Test - public void parseAddingOverlayWithUnnamedBaselineProfile() { + public void parseAddingOverlayWithUnnamedBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728441200.plus_profile=16,+(q.r.s=four,l.m=16)"); Assert.assertEquals(sd, new ScalingDirective("plus_profile", 16, 1728441200L, WorkforceProfiles.BASELINE_NAME, new ProfileOverlay.Adding( @@ -123,7 +127,7 @@ public void parseAddingOverlayWithUnnamedBaselineProfile() { } @Test - public void parseAddingOverlayWithBaselineProfile() { + public void parseAddingOverlayWithBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728443640.plus_profile=16,baseline()+(q.r=five,l.m=12)"); Assert.assertEquals(sd, new ScalingDirective("plus_profile", 16, 1728443640L, WorkforceProfiles.BASELINE_NAME, new ProfileOverlay.Adding( @@ -131,19 +135,47 @@ public void parseAddingOverlayWithBaselineProfile() { } @Test - public void parseRemovingOverlayWithUnnamedBaselineProfile() { + public void parseRemovingOverlayWithUnnamedBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728448521.extra_profile=0,-(a.b, c.d)"); Assert.assertEquals(sd, new ScalingDirective("extra_profile", 0, 1728448521L, WorkforceProfiles.BASELINE_NAME, new ProfileOverlay.Removing(Lists.newArrayList("a.b", "c.d")))); } @Test - public void parseRemovingOverlayWithBaselineProfile() { + public void parseRemovingOverlayWithBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("4.extra_profile=9,baseline()-(a.b, c.d)"); Assert.assertEquals(sd, new ScalingDirective("extra_profile", 9, 4L, WorkforceProfiles.BASELINE_NAME, new ProfileOverlay.Removing(Lists.newArrayList("a.b", "c.d")))); } + @Test + public void parseProfileIdTooLong() throws ScalingDirectiveParser.InvalidSyntaxException { + BiFunction fmtRemovingOverlaySyntax = (profileId, basisProfileId) -> { + return "1728449000." + profileId + "=99," + basisProfileId + "-(foo,bar,baz)"; + }; + String alphabet = IntStream.rangeClosed('a', 'z').collect(StringBuilder::new, StringBuilder::appendCodePoint, StringBuilder::append).toString(); + String tooLongId = alphabet + alphabet.toUpperCase() + alphabet + alphabet.toUpperCase(); + Assert.assertTrue(tooLongId.length() > ScalingDirectiveParser.MAX_PROFILE_IDENTIFIER_LENGTH); + + final int atMaxLen = ScalingDirectiveParser.MAX_PROFILE_IDENTIFIER_LENGTH; + final int beyondMaxLen = ScalingDirectiveParser.MAX_PROFILE_IDENTIFIER_LENGTH + 1; + String notTooLongDirective1 = fmtRemovingOverlaySyntax.apply(tooLongId.substring(0, atMaxLen), "some_profile"); + String notTooLongDirective2 = fmtRemovingOverlaySyntax.apply("new_profile", tooLongId.substring(0, atMaxLen)); + String notTooLongDirective3 = fmtRemovingOverlaySyntax.apply(tooLongId.substring(0, atMaxLen), tooLongId.substring(1, atMaxLen + 1)); + + for (String directiveStr : new String[] { notTooLongDirective1, notTooLongDirective2, notTooLongDirective3 }) { + Assert.assertNotNull(parser.parse(directiveStr)); + } + + String tooLongDirective1 = fmtRemovingOverlaySyntax.apply(tooLongId.substring(0, beyondMaxLen), "some_profile"); + String tooLongDirective2 = fmtRemovingOverlaySyntax.apply("new_profile", tooLongId.substring(0, beyondMaxLen)); + String tooLongDirective3 = fmtRemovingOverlaySyntax.apply(tooLongId.substring(0, beyondMaxLen), tooLongId.substring(1, beyondMaxLen + 1)); + + Arrays.stream(new String[] { tooLongDirective1, tooLongDirective2, tooLongDirective3 }).forEach(directiveStr -> { + Assert.assertThrows(ScalingDirectiveParser.InvalidSyntaxException.class, () -> parser.parse(directiveStr)); + }); + } + @DataProvider(name = "funkyButValidDirectives") public String[][] validDirectives() { return new String[][]{ @@ -173,7 +205,7 @@ public String[][] validDirectives() { expectedExceptions = {}, dataProvider = "funkyButValidDirectives" ) - public void parseValidDirectives(String directive) { + public void parseValidDirectives(String directive) throws ScalingDirectiveParser.InvalidSyntaxException { Assert.assertNotNull(parser.parse(directive)); } @@ -195,10 +227,59 @@ public String[][] validDirectivesForRoundTrip() { expectedExceptions = {}, dataProvider = "validDirectivesToRoundTrip" ) - public void roundTripAsStringAfterParse(String directive) { + public void roundTripAsStringAfterParse(String directive) throws ScalingDirectiveParser.InvalidSyntaxException { Assert.assertEquals(ScalingDirectiveParser.asString(parser.parse(directive)), directive); } + @DataProvider(name = "validDirectivesWithOverlayPlaceholder") + public String[][] validDirectivesWithOverlayPlaceholder() { + return new String[][]{ + { "6.extra_profile=9,the_basis+(...)" }, + { "6.extra_profile=9;the_basis+(...)" }, + { "6.extra_profile=9,the_basis-(...)" }, + { "6.extra_profile=9;the_basis-(...)" } + }; + } + + @Test( + expectedExceptions = ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition.class, + dataProvider = "validDirectivesWithOverlayPlaceholder" + ) + public void parseDirectivesWithPlaceholderThrowsOverlayPlaceholderNeedsDefinition(String directive) throws ScalingDirectiveParser.InvalidSyntaxException { + Assert.assertEquals(ScalingDirectiveParser.asString(parser.parse(directive)), directive); + } + + @DataProvider(name = "overlayPlaceholderDirectivesWithCompletionDefAndEquivalent") + public String[][] overlayPlaceholderDirectivesWithCompletionDefAndEquivalent() { + return new String[][]{ + { "6.extra_profile=9,the_basis+(...)", "a=7,m=sixteen", "6.extra_profile=9,the_basis+(a=7,m=sixteen)" }, + { "6.extra_profile=9,the_basis+(...)", "a=7;m=sixteen", "6.extra_profile=9,the_basis+(a=7%3Bm%3Dsixteen)" }, // sep mismatch, so val == "7;m=sixteen" + { "6.extra_profile=9,the_basis+(...)", "a.b.c=7,l.m=sixteen%2C%20again", "6.extra_profile=9,the_basis+(a.b.c=7,l.m=sixteen%2C%20again)" }, + { "6.extra_profile=9;the_basis+(...)", "a=7,m=sixteen", "6.extra_profile=9;the_basis+(a=7%2Cm%3Dsixteen)" }, // sep mismatch, so val == "7,m=sixteen" + { "6.extra_profile=9;the_basis+(...)", "a=7;m=sixteen", "6.extra_profile=9;the_basis+(a=7;m=sixteen)" }, + { "6.extra_profile=9;the_basis+(...)", "a.b.c=7;l.m=sixteen%2C%20again", "6.extra_profile=9;the_basis+(a.b.c=7;l.m=sixteen%2C%20again)" }, + { "6.extra_profile=9,the_basis-(...)", "a.b,x.y.z", "6.extra_profile=9,the_basis-(a.b,x.y.z)" }, + { "6.extra_profile=9,the_basis-(...)", "x,y.z", "6.extra_profile=9,the_basis-(x,y.z)" }, + { "6.extra_profile=9;the_basis-(...)", "x;y.z", "6.extra_profile=9;the_basis-(x;y.z)" }, + { "6.extra_profile=9;the_basis-(...)", "a.b;x.y.z", "6.extra_profile=9;the_basis-(a.b;x.y.z)" } + }; + } + + @Test( + expectedExceptions = {}, + dataProvider = "overlayPlaceholderDirectivesWithCompletionDefAndEquivalent" + ) + public void verifyPlaceholderEquivalence(String directiveWithPlaceholder, String overlayDefinition, String equivDirective) + throws ScalingDirectiveParser.InvalidSyntaxException { + try { + parser.parse(directiveWithPlaceholder); + Assert.fail("Expected `ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition` due to the placeholder in the directive"); + } catch (ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition needsDefinition) { + ScalingDirective directive = needsDefinition.retryParsingWithDefinition(overlayDefinition); + Assert.assertEquals(directive, parser.parse(equivDirective)); + } + } + @DataProvider(name = "invalidDirectives") public String[][] invalidDirectives() { return new String[][] { @@ -234,6 +315,14 @@ public String[][] invalidDirectives() { { "1728439210.new_profile=16,bar+(a=7,)" }, { "1728439210.new_profile=16;bar+(a=7;)" }, + // adding: overlay placeholder may not be used with key-value pairs: + { "1728439210.new_profile=16,bar+(a=7,...)" }, + { "1728439210.new_profile=16,bar+(...,b=4)" }, + { "1728439210.new_profile=16,bar+(a=7,...,b=4)" }, + { "1728439210.new_profile=16;bar+(a=7;...)" }, + { "1728439210.new_profile=16;bar+(...;b=4)" }, + { "1728439210.new_profile=16;bar+(a=7;...;b=4)" }, + // removing: invalid set-point + missing token examples: { "1728436436.other_profile=-9,my_profile-(x)" }, { "1728436436.other_profile=69,my_profile-(x" }, @@ -251,6 +340,14 @@ public String[][] invalidDirectives() { { "1728436436.other_profile=69,my_profile-(x,)" }, { "1728436436.other_profile=69;my_profile-(x;)" }, + // removing: overlay placeholder may not be used with keys: + { "1728436436.other_profile=69,my_profile-(x,...)" }, + { "1728436436.other_profile=69,my_profile-(...,z)" }, + { "1728436436.other_profile=69,my_profile-(x,...,z)" }, + { "1728436436.other_profile=69;my_profile-(x;...)" }, + { "1728436436.other_profile=69;my_profile-(...;z)" }, + { "1728436436.other_profile=69;my_profile-(x;...;z)" }, + // removing: seemingly separator mismatch, but in fact the NOT-separator is illegal in a key (e.g. "x;y"): { "1728436436.other_profile=69,my_profile-(x;y)" }, { "1728436436.other_profile=69;my_profile-(x,y)" }, @@ -260,10 +357,42 @@ public String[][] invalidDirectives() { } @Test( - expectedExceptions = ScalingDirectiveParser.MalformedDirectiveException.class, + expectedExceptions = ScalingDirectiveParser.InvalidSyntaxException.class, dataProvider = "invalidDirectives" ) - public void parseInvalidDirectives(String directive) { + public void parseInvalidDirectives(String directive) throws ScalingDirectiveParser.InvalidSyntaxException { parser.parse(directive); } + + @DataProvider(name = "overlayPlaceholderDirectivesWithInvalidCompletionDef") + public String[][] overlayPlaceholderDirectivesWithInvalidCompletionDef() { + return new String[][]{ + { "6.extra_profile=9,the_basis+(...)", "..." }, + { "6.extra_profile=9;the_basis+(...)", "..." }, + { "6.extra_profile=9,the_basis+(...)", "a=7," }, + { "6.extra_profile=9;the_basis+(...)", "a=7;" }, + { "6.extra_profile=9,the_basis+(...)", "a.b.c,l.m.n" }, + { "6.extra_profile=9;the_basis+(...)", "a.b.c;l.m.n" }, + { "6.extra_profile=9,the_basis-(...)", "..." }, + { "6.extra_profile=9;the_basis-(...)", "..." }, + { "6.extra_profile=9,the_basis-(...)", "a.b," }, + { "6.extra_profile=9;the_basis-(...)", "a.b;" }, + { "6.extra_profile=9,the_basis-(...)", "x=foo,y.z=bar" }, + { "6.extra_profile=9;the_basis-(...)", "x=foo;y.z=bar" } + }; + } + + @Test( + expectedExceptions = ScalingDirectiveParser.InvalidSyntaxException.class, + dataProvider = "overlayPlaceholderDirectivesWithInvalidCompletionDef" + ) + public void verifyPlaceholderDefIsInvalid(String directiveWithPlaceholder, String invalidOverlayDefinition) + throws ScalingDirectiveParser.InvalidSyntaxException { + try { + parser.parse(directiveWithPlaceholder); + Assert.fail("Expected `ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition` due to the placeholder in the directive"); + } catch (ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition needsDefinition) { + Assert.assertNotNull(needsDefinition.retryParsingWithDefinition(invalidOverlayDefinition)); + } + } } From 617f7766a987f37845c0c8fff45ef07bdccafddd Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Tue, 5 Nov 2024 23:16:01 -0800 Subject: [PATCH 07/11] Add (most) javadoc --- .../dynamic/FsScalingDirectiveSource.java | 70 +++++++++++------ .../temporal/dynamic/ProfileDerivation.java | 8 ++ .../temporal/dynamic/ProfileOverlay.java | 48 ++++++++---- .../temporal/dynamic/ScalingDirective.java | 7 ++ .../dynamic/ScalingDirectiveSource.java | 3 +- .../temporal/dynamic/StaffingDeltas.java | 10 ++- .../temporal/dynamic/WorkerProfile.java | 1 + .../temporal/dynamic/WorkforcePlan.java | 43 ++++++++--- .../temporal/dynamic/WorkforceProfiles.java | 45 ++++++++--- .../temporal/dynamic/WorkforceStaffing.java | 57 ++++++++++---- .../dynamic/FsScalingDirectiveSourceTest.java | 77 ++++++++++++------- .../dynamic/ProfileDerivationTest.java | 9 ++- .../temporal/dynamic/ProfileOverlayTest.java | 25 +++--- .../dynamic/ScalingDirectiveParserTest.java | 62 +++++++-------- .../temporal/dynamic/WorkforcePlanTest.java | 42 ++++++---- .../dynamic/WorkforceStaffingTest.java | 10 ++- 16 files changed, 336 insertions(+), 181 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java index 395aedadaf7..9901c1e14ca 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java @@ -17,7 +17,6 @@ package org.apache.gobblin.temporal.dynamic; -import com.google.common.base.Charsets; import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; @@ -26,6 +25,7 @@ import java.util.Map; import java.util.Optional; +import com.google.common.base.Charsets; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.IOUtils; @@ -34,6 +34,13 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +/** + * A {@link ScalingDirectiveSource} that reads {@link ScalingDirective}s from a {@link FileSystem} directory, where each directive is the name + * of a single file inside the directory. Directives too long for one filename path component MUST use the + * {@link ScalingDirectiveParser#OVERLAY_DEFINITION_PLACEHOLDER} syntax and write their {@link ProfileDerivation} overlay as the file's data/content. + * Within-length scaling directives are no-data, zero-length files. When backed by HDFS, reading such zero-length scaling directive files is a + * NameNode-only operation, while their metadata-only nature additionally conserves NN object count/quota. + */ @Slf4j public class FsScalingDirectiveSource implements ScalingDirectiveSource { private final FileSystem fileSystem; @@ -41,17 +48,32 @@ public class FsScalingDirectiveSource implements ScalingDirectiveSource { private final Optional optErrorsPath; private final ScalingDirectiveParser parser = new ScalingDirectiveParser(); - public FsScalingDirectiveSource(FileSystem fileSystem, String directivesDirPath, Optional optErrorDirPath) { + /** Read from `directivesDirPath` of `fileSystem`, and optionally move invalid/rejected directives to `optErrorsDirPath` */ + public FsScalingDirectiveSource(FileSystem fileSystem, String directivesDirPath, Optional optErrorsDirPath) { this.fileSystem = fileSystem; this.dirPath = new Path(directivesDirPath); - this.optErrorsPath = optErrorDirPath.map(Path::new); + this.optErrorsPath = optErrorsDirPath.map(Path::new); } - // TODO: describe purpose of constraint (to preclude late insertion/edits of the directives stream) - - // verify and only return directives whose stated (in filename) timestamp order matches `FileStatus` modtime order + /** + * @return all valid (parseable, in-order) scaling directives currently in the directory, ordered by ascending modtime + * + * Ignore invalid directives, and, when `optErrorsDirPath` was provided to the ctor, acknowledge each by moving it to a separate "errors" directory. + * Regardless, always swallow {@link ScalingDirectiveParser.InvalidSyntaxException}. + * + * Like un-parseable directives, so too are out-of-order directives invalid. This prevents late/out-of-order insertion and/or edits to the directives + * stream. Each directive contains its own {@link ScalingDirective#getTimestampEpochMillis()} stated in its filename. Later-modtime directives are + * rejected when directive-timestamp-order does not match {@link FileStatus} modtime order. In the case of a modtime tie, the directive with the + * alphabetically-later filename is rejected. + * + * NOTE: This returns ALL known directives, even those already returned by a prior invocation. + * + * @throws IOException when unable to read the directory (or file data, in the case of an overlay definition placeholder) + */ @Override public List getScalingDirectives() throws IOException { List> directiveWithFileStatus = new ArrayList<>(); + // to begin, just parse w/o worrying about ordering... that comes next for (FileStatus fileStatus : fileSystem.listStatus(dirPath)) { if (!fileStatus.isFile()) { log.warn("Ignoring non-file object: " + fileStatus); @@ -60,7 +82,7 @@ public List getScalingDirectives() throws IOException { String fileName = fileStatus.getPath().getName(); try { try { - directiveWithFileStatus.add(new ImmutablePair<>(parseScalingDirective(fileName), fileStatus)); + directiveWithFileStatus.add(new ImmutablePair<>(parser.parse(fileName), fileStatus)); } catch (ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition needsDefinition) { // directive used placeholder syntax to indicate the overlay definition resides inside its file... so open the file to load that def log.info("Loading overlay definition for directive {{" + fileName + "}} from: " + fileStatus); @@ -74,15 +96,15 @@ public List getScalingDirectives() throws IOException { } } - // verify and only return directives whose ordering of stated (in filename) timestamp matches `FileStatus` modtime order + // verify ordering: only return directives whose stated timestamp ordering (of filename prefix) matches `FileStatus` modtime order List directives = new ArrayList<>(); - // NOTE: for deterministic total-ordering, sort by path, rather than by timestamp, in case of modtime tie (given only secs granularity) + // NOTE: for deterministic total-ordering, sort by path, rather than by timestamp, in case of modtime tie (given only millisecs granularity) directiveWithFileStatus.sort(Comparator.comparing(p -> p.getValue().getPath())); long latestValidModTime = -1; for (Map.Entry entry : directiveWithFileStatus) { long thisModTime = entry.getValue().getModificationTime(); - if (thisModTime < latestValidModTime) { // do NOT reject equal (non-increasing) modtime, given granularity of epoch seconds - log.warn("Ignoring out-of-order scaling directive " + entry.getKey() + " since FS modTime " + thisModTime + " precedes last observed " + if (thisModTime <= latestValidModTime) { // when equal (non-increasing) modtime: reject alphabetically-later filename (path) + log.warn("Ignoring out-of-order scaling directive " + entry.getKey() + " since FS modTime " + thisModTime + " NOT later than last observed " + latestValidModTime + ": " + entry.getValue()); optAcknowledgeError(entry.getValue(), "out-of-order"); } else { @@ -93,32 +115,32 @@ public List getScalingDirectives() throws IOException { return directives; } - // ack error by moving the bad/non-directive to a separate errors dir - protected void optAcknowledgeError(FileStatus fileStatus, String desc) { + /** "acknowledge" the rejection of an invalid directive by moving it to a separate "errors" dir (when `optErrorsDirPath` was given to the ctor) */ + protected void optAcknowledgeError(FileStatus invalidDirectiveFileStatus, String desc) { this.optErrorsPath.ifPresent(errorsPath -> - moveToErrors(fileStatus, errorsPath, desc) + moveDirectiveToDir(invalidDirectiveFileStatus, errorsPath, desc) ); } - // move broken/ignored directives into a separate directory, as an observability-enhancing ack of its rejection - protected void moveToErrors(FileStatus badDirectiveStatus, Path errorsPath, String desc) { - Path badDirectivePath = badDirectiveStatus.getPath(); + /** + * move `invalidDirectiveFileStatus` to a designated `destDirPath`, with the reason for moving (e.g. the error) described in `desc`. + * This is used to promote observability by acknowledging invalid, rejected directives + */ + protected void moveDirectiveToDir(FileStatus invalidDirectiveFileStatus, Path destDirPath, String desc) { + Path invalidDirectivePath = invalidDirectiveFileStatus.getPath(); try { - if (!this.fileSystem.rename(badDirectivePath, new Path(errorsPath, badDirectivePath.getName()))) { + if (!this.fileSystem.rename(invalidDirectivePath, new Path(destDirPath, invalidDirectivePath.getName()))) { throw new RuntimeException(); // unclear how to obtain more info about such a failure } } catch (IOException e) { - log.warn("Failed to move " + desc + " directive {{" + badDirectiveStatus.getPath() + "}} to '" + errorsPath + "'... leaving in place", e); + log.warn("Failed to move " + desc + " directive {{" + invalidDirectiveFileStatus.getPath() + "}} to '" + destDirPath + "'... leaving in place", e); } catch (RuntimeException e) { - log.warn("Failed to move " + desc + " directive {{" + badDirectiveStatus.getPath() + "}} to '" + errorsPath + "' [unknown reason]... leaving in place"); + log.warn("Failed to move " + desc + " directive {{" + invalidDirectiveFileStatus.getPath() + "}} to '" + destDirPath + + "' [unknown reason]... leaving in place", e); } } - private ScalingDirective parseScalingDirective(String fileName) - throws ScalingDirectiveParser.InvalidSyntaxException, ScalingDirectiveParser.OverlayPlaceholderNeedsDefinition { - return parser.parse(fileName); - } - + /** @return all contents of `path` as a single (UTF-8) `String` */ protected String slurpFileAsString(Path path) throws IOException { try (InputStream is = this.fileSystem.open(path)) { return IOUtils.toString(is, Charsets.UTF_8); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java index 013e9139581..5eafe8a4332 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java @@ -25,8 +25,14 @@ import lombok.Getter; +/** + * Defines a new {@link WorkerProfile} by evolving from another profile, the basis. Such evolution creates a new immutable profile through + * {@link ProfileOverlay}, which either adds or removes properties from the basis profile's definition. That basis profile must already exist. + */ @Data public class ProfileDerivation { + + /** Flags when the basis profile was NOT found */ public static class UnknownBasisException extends Exception { @Getter private final String name; @@ -39,6 +45,7 @@ public UnknownBasisException(String basisName) { private final String basisProfileName; private final ProfileOverlay overlay; + /** @return a new profile definition through evolution from the basis profile, which is to be obtained via `basisResolver` */ public Config formulateConfig(Function> basisResolver) throws UnknownBasisException { Optional optProfile = basisResolver.apply(basisProfileName); if (!optProfile.isPresent()) { @@ -48,6 +55,7 @@ public Config formulateConfig(Function> basisRes } } + /** @return the canonical display name of {@link #getBasisProfileName()} for tracing/debugging */ public String renderName() { return WorkforceProfiles.renderName(this.basisProfileName); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileOverlay.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileOverlay.java index 45434d74fc4..64b5d8ec30b 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileOverlay.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileOverlay.java @@ -18,6 +18,7 @@ package org.apache.gobblin.temporal.dynamic; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -27,15 +28,20 @@ import com.typesafe.config.Config; import com.typesafe.config.ConfigValueFactory; import lombok.Data; +import lombok.RequiredArgsConstructor; +/** Alt. forms of profile overlay to evolve one profile {@link Config} into another. Two overlays may be combined hierarchically into a new overlay. */ public interface ProfileOverlay { + /** @return a new, evolved {@link Config}, by application of this overlay */ Config applyOverlay(Config config); + /** @return a new overlay, by combining this overlay *over* another */ ProfileOverlay over(ProfileOverlay other); + /** A key-value pair/duple */ @Data class KVPair { private final String key; @@ -43,11 +49,16 @@ class KVPair { } + /** An overlay to evolve any profile by adding key-value pairs */ @Data - // TODO: variadic ctor/factory + @RequiredArgsConstructor // explicit, due to second, variadic ctor class Adding implements ProfileOverlay { private final List additionPairs; + public Adding(KVPair... kvPairs) { + this(Arrays.asList(kvPairs)); + } + @Override public Config applyOverlay(Config config) { return additionPairs.stream().sequential().reduce(config, @@ -70,18 +81,23 @@ public ProfileOverlay over(ProfileOverlay other) { } else if (other instanceof Combo) { Combo otherCombo = (Combo) other; return Combo.normalize((Adding) this.over(otherCombo.getAdding()), otherCombo.getRemoving()); - } else { + } else { // should NEVER happen! throw new IllegalArgumentException("unknown derived class of type '" + other.getClass().getName() + "': " + other); } } } + /** An overlay to evolve any profile by removing named keys */ @Data - // TODO: variadic ctor/factory + @RequiredArgsConstructor // explicit, due to second, variadic ctor class Removing implements ProfileOverlay { private final List removalKeys; + public Removing(String... keys) { + this(Arrays.asList(keys)); + } + @Override public Config applyOverlay(Config config) { return removalKeys.stream().sequential().reduce(config, @@ -103,33 +119,25 @@ public ProfileOverlay over(ProfileOverlay other) { } else if (other instanceof Combo) { Combo otherCombo = (Combo) other; return Combo.normalize(otherCombo.getAdding(), (Removing) this.over(otherCombo.getRemoving())); - } else { + } else { // should NEVER happen! throw new IllegalArgumentException("unknown derived class of type '" + other.getClass().getName() + "': " + other); } } } + /** An overlay to evolve any profile by adding key-value pairs while also removing named keys */ @Data class Combo implements ProfileOverlay { private final Adding adding; private final Removing removing; - // merely restrict access modifier from `public` to `protected`, as not meant to be instantiated outside this enclosing interface + /** restricted-access ctor: instead use {@link Combo#normalize(Adding, Removing)} */ private Combo(Adding adding, Removing removing) { this.adding = adding; this.removing = removing; } - protected static Combo normalize(Adding toAdd, Removing toRemove) { - // pre-remove any in `toAdd` that are also in `toRemove`... yet still maintain them in `toRemove`, in case the eventual `Config` "basis" also has any - Set removeKeysLookup = toRemove.getRemovalKeys().stream().collect(Collectors.toSet()); - List unmatchedAdditionPairs = toAdd.getAdditionPairs().stream().sequential().filter(additionPair -> - !removeKeysLookup.contains(additionPair.getKey()) - ).collect(Collectors.toList()); - return new Combo(new Adding(unmatchedAdditionPairs), new Removing(new ArrayList<>(removeKeysLookup))); - } - @Override public Config applyOverlay(Config config) { return adding.applyOverlay(removing.applyOverlay(config)); @@ -144,9 +152,19 @@ public ProfileOverlay over(ProfileOverlay other) { } else if (other instanceof Combo) { Combo otherCombo = (Combo) other; return Combo.normalize((Adding) this.adding.over(otherCombo.getAdding()), (Removing) this.removing.over(otherCombo.getRemoving())); - } else { + } else { // should NEVER happen! throw new IllegalArgumentException("unknown derived class of type '" + other.getClass().getName() + "': " + other); } } + + /** @return a `Combo` overlay, by combining an `Adding` overlay with a `Removing` overlay */ + protected static Combo normalize(Adding toAdd, Removing toRemove) { + // pre-remove any in `toAdd` that are also in `toRemove`... yet still maintain all in `toRemove`, in case also in the eventual `Config` "basis" + Set removeKeysLookup = toRemove.getRemovalKeys().stream().collect(Collectors.toSet()); + List unmatchedAdditionPairs = toAdd.getAdditionPairs().stream().sequential().filter(additionPair -> + !removeKeysLookup.contains(additionPair.getKey()) + ).collect(Collectors.toList()); + return new Combo(new Adding(unmatchedAdditionPairs), new Removing(new ArrayList<>(removeKeysLookup))); + } } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java index 5a4da3d4d38..a95bef7541a 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java @@ -22,6 +22,11 @@ import lombok.RequiredArgsConstructor; +/** + * Core abstraction to model scaling adjustment, which originates at a given moment in time. A directive provides a set point for a given worker profile. + * The set point is the number of instances presently desired for that profile. When naming a heretofore unknown worker profile, the directive MUST also + * define that new profile through a {@link ProfileDerivation} that references a known profile. Known worker profiles MUST NOT be redefined. + */ @Data @RequiredArgsConstructor public class ScalingDirective { @@ -30,6 +35,7 @@ public class ScalingDirective { private final long timestampEpochMillis; private final Optional optDerivedFrom; + /** Create a set-point-only directive (for a known profile, with no {@link ProfileDerivation}) */ public ScalingDirective(String profileName, int setPoint, long timestampEpochMillis) { this(profileName, setPoint, timestampEpochMillis, Optional.empty()); } @@ -38,6 +44,7 @@ public ScalingDirective(String profileName, int setPoint, long timestampEpochMil this(profileName, setPoint, timestampEpochMillis, Optional.of(new ProfileDerivation(basisProfileName, overlay))); } + /** @return the canonical display name (of {@link #getProfileName()}) for tracing/debugging */ public String renderName() { return WorkforceProfiles.renderName(this.profileName); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveSource.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveSource.java index 98f1632e6f1..1b0f79e78d6 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveSource.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveSource.java @@ -21,7 +21,8 @@ import java.util.List; +/** An opaque source of {@link org.apache.gobblin.temporal.dynamic.ScalingDirective}s */ public interface ScalingDirectiveSource extends Cloneable { - // TODO - document! (impl may choose to give only newer directives, not previously returned... or to return them all) + /** @return {@link ScalingDirective}s - an impl. may choose to return all known directives or to give only newer directives than previously returned */ List getScalingDirectives() throws IOException; } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/StaffingDeltas.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/StaffingDeltas.java index 0cf0a8a5a94..47b92dde40e 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/StaffingDeltas.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/StaffingDeltas.java @@ -21,16 +21,22 @@ import lombok.Data; +/** Staffing set point {@link ProfileDelta}s for multiple {@link WorkerProfile}s */ @Data public class StaffingDeltas { + /** + * Difference for a {@link WorkerProfile}'s staffing set point (e.g. between desired and current levels). Positive `delta` reflects increase, + * while negative, a decrease. + */ @Data public static class ProfileDelta { private final WorkerProfile profile; private final int delta; private final long setPointProvenanceEpochMillis; - public boolean isUnchanged() { - return delta == 0; + /** @return whether {@link #getDelta()} is non-zero */ + public boolean isChange() { + return delta != 0; } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkerProfile.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkerProfile.java index 385bf7e6d5f..bf1f1d2e099 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkerProfile.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkerProfile.java @@ -21,6 +21,7 @@ import lombok.Data; +/** A named worker {@link Config} */ @Data public class WorkerProfile { private final String name; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java index 76b5a44a497..ec82ae0cd9d 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java @@ -17,19 +17,27 @@ package org.apache.gobblin.temporal.dynamic; -import com.google.common.annotations.VisibleForTesting; import java.util.List; import java.util.Optional; import java.util.function.Consumer; +import javax.annotation.concurrent.ThreadSafe; +import com.google.common.annotations.VisibleForTesting; import com.typesafe.config.Config; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +/** + * Stateful class to maintain the dynamically scalable workforce plan for {@link WorkerProfile}s with a {@link WorkforceStaffing} set point + * for each. The plan evolves through incremental revision by {@link ScalingDirective}s, while {@link #calcStaffingDeltas(WorkforceStaffing)} + * reports {@link StaffingDeltas} between the current plan and another alternative (e.g. current level of) {@link WorkforceStaffing}. + */ @Slf4j +@ThreadSafe public class WorkforcePlan { + /** Common baseclass for illegal plan revision */ public static class IllegalRevisionException extends Exception { @Getter private final ScalingDirective directive; private IllegalRevisionException(ScalingDirective directive, String msg) { @@ -37,6 +45,7 @@ private IllegalRevisionException(ScalingDirective directive, String msg) { this.directive = directive; } + /** Illegal revision: directive arrived out of {@link ScalingDirective#getTimestampEpochMillis()} order */ public static class OutdatedDirective extends IllegalRevisionException { protected OutdatedDirective(ScalingDirective directive, long lastRevisionEpochMillis) { super(directive, "directive for profile '" + directive.renderName() + "' precedes last revision at " @@ -44,6 +53,7 @@ protected OutdatedDirective(ScalingDirective directive, long lastRevisionEpochMi } } + /** Illegal revision: redefinition of a known worker profile */ public static class Redefinition extends IllegalRevisionException { protected Redefinition(ScalingDirective directive, ProfileDerivation proposedDerivation) { super(directive, "profile '" + directive.renderName() + "' already exists, so may not be redefined on the basis of '" @@ -51,12 +61,14 @@ protected Redefinition(ScalingDirective directive, ProfileDerivation proposedDer } } + /** Illegal revision: set point for an unknown worker profile */ public static class UnrecognizedProfile extends IllegalRevisionException { protected UnrecognizedProfile(ScalingDirective directive) { super(directive, "unrecognized profile reference '" + directive.renderName() + "': " + directive); } } + /** Illegal revision: worker profile evolution from an unknown basis profile */ public static class UnknownBasis extends IllegalRevisionException { protected UnknownBasis(ScalingDirective directive, ProfileDerivation.UnknownBasisException ube) { super(directive, "profile '" + directive.renderName() + "' may not be defined on the basis of an unknown profile '" @@ -69,16 +81,19 @@ protected UnknownBasis(ScalingDirective directive, ProfileDerivation.UnknownBasi private final WorkforceStaffing staffing; @Getter private volatile long lastRevisionEpochMillis; + /** create new plan with `baselineConfig` with `initialSetPoint` of the initial, baseline worker profile */ public WorkforcePlan(Config baselineConfig, int initialSetPoint) { this.profiles = WorkforceProfiles.withBaseline(baselineConfig); this.staffing = WorkforceStaffing.initialize(initialSetPoint); this.lastRevisionEpochMillis = 0; } + /** @return how many worker profiles known to the plan, including the baseline */ public int getNumProfiles() { return profiles.size(); } + /** revise the plan with a new {@link ScalingDirective} or throw {@link IllegalRevisionException} */ public synchronized void revise(ScalingDirective directive) throws IllegalRevisionException { String name = directive.getProfileName(); if (this.lastRevisionEpochMillis >= directive.getTimestampEpochMillis()) { @@ -98,25 +113,30 @@ public synchronized void revise(ScalingDirective directive) throws IllegalRevisi throw new IllegalRevisionException.UnknownBasis(directive, ube); } } - // TODO - make idempotent, as re-attempts after failure between `addProfile` and `reviseStaffing` would fail with `IllegalRevisionException.Redefinition` - // adjust the set-point now that either a new profile is defined OR the profile already existed + // TODO - make idempotent, since any retry attempts after a failure between `addProfile` and `reviseStaffing` would henceforth fail with + // `IllegalRevisionException.Redefinition`, despite us wishing to adjust the set point now that the new profile has been defined... + // how to ensure the profile def is the same / unchanged? (e.g. compare full profile `Config` equality)? this.staffing.reviseStaffing(name, directive.getSetPoint(), directive.getTimestampEpochMillis()); this.lastRevisionEpochMillis = directive.getTimestampEpochMillis(); } } - /** atomic bulk revision - * - * !!!!requires sorted order of directives by timestamp!!!! + /** + * Performs atomic bulk revision while enforcing `directives` ordering by {@link ScalingDirective#getTimestampEpochMillis()} * + * This version catches {@link IllegalRevisionException}s, logging a warning message for any before continuing to process subsequent directives. */ public synchronized void reviseWhenNewer(List directives) { reviseWhenNewer(directives, ire -> { log.warn("Failure: ", ire); }); } + /** + * Performs atomic bulk revision while enforcing `directives` ordering by {@link ScalingDirective#getTimestampEpochMillis()} + * + * This version catches {@link IllegalRevisionException}s, feeding any to `illegalRevisionHandler` before continuing to process subsequent directives. + */ public synchronized void reviseWhenNewer(List directives, Consumer illegalRevisionHandler) { directives.stream().sequential() - .filter(directive -> directive.getTimestampEpochMillis() > this.lastRevisionEpochMillis) .forEach(directive -> { try { revise(directive); @@ -126,21 +146,24 @@ public synchronized void reviseWhenNewer(List directives, Cons }); } - /** @returns diff of {@link StaffingDeltas} of this, current {@link WorkforcePlan} against some `reference` {@link WorkforceStaffing} */ - public synchronized StaffingDeltas calcStaffingDeltas(WorkforceStaffing reference) { - return staffing.calcDeltas(reference, profiles); + /** @return diff of {@link StaffingDeltas} between this, current {@link WorkforcePlan} and some `altStaffing` (e.g. current) {@link WorkforceStaffing} */ + public synchronized StaffingDeltas calcStaffingDeltas(WorkforceStaffing altStaffing) { + return staffing.calcDeltas(altStaffing, profiles); } + /** @return [for testing/debugging] the current staffing set point for the {@link WorkerProfile} named `profileName`, when it exists */ @VisibleForTesting Optional peepStaffing(String profileName) { return staffing.getStaffing(profileName); } + /** @return [for testing/debugging] the {@link WorkerProfile} named `profileName` or throws {@link WorkforceProfiles.UnknownProfileException} */ @VisibleForTesting WorkerProfile peepProfile(String profileName) throws WorkforceProfiles.UnknownProfileException { return profiles.getOrThrow(profileName); } + /** @return [for testing/debugging] the baseline {@link WorkerProfile} - it should NEVER {@link WorkforceProfiles.UnknownProfileException} */ @VisibleForTesting WorkerProfile peepBaselineProfile() throws WorkforceProfiles.UnknownProfileException { return profiles.getOrThrow(WorkforceProfiles.BASELINE_NAME); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java index dbe9a597306..5cd3bbab27c 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java @@ -20,42 +20,68 @@ import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; +import javax.annotation.concurrent.ThreadSafe; import com.typesafe.config.Config; +import lombok.Getter; +/** A collection of known {@link WorkerProfile}s, also offering name -> profile resolution for {@link ProfileDerivation} */ +@ThreadSafe public class WorkforceProfiles implements Function> { + + /** Indicates `profileName` NOT found */ + public static class UnknownProfileException extends RuntimeException { + @Getter + private final String profileName; + + public UnknownProfileException(String profileName) { + super("named '" + WorkforceProfiles.renderName(profileName) + "'"); + this.profileName = profileName; + } + } + + public static final String BASELINE_NAME = ""; public static final String BASELINE_NAME_RENDERING = "<>"; + /** @return the canonical display name for tracing/debugging, with special handling for {@link #BASELINE_NAME} */ public static String renderName(String name) { return name.equals(BASELINE_NAME) ? BASELINE_NAME_RENDERING : name; } - public static class UnknownProfileException extends RuntimeException { - public UnknownProfileException(String profileName) { - super("named '" + WorkforceProfiles.renderName(profileName) + "'"); - } - } - private final ConcurrentHashMap profileByName; + /** restricted-access ctor: instead use {@link #withBaseline(Config)} */ private WorkforceProfiles() { this.profileByName = new ConcurrentHashMap<>(); } + /** @return a new instance with `baselineConfig` as the "baseline profile" */ public static WorkforceProfiles withBaseline(Config baselineConfig) { WorkforceProfiles profiles = new WorkforceProfiles(); profiles.addProfile(new WorkerProfile(BASELINE_NAME, baselineConfig)); return profiles; } + /** Add a new, previously unknown {@link WorkerProfile} or throw `RuntimeException` on any attempt to add/redefine a previously known profile */ + public void addProfile(WorkerProfile profile) { + if (profileByName.putIfAbsent(profile.getName(), profile) != null) { + throw new RuntimeException("profile '" + WorkforceProfiles.renderName(profile.getName()) + "' already exists!"); + } + } + + /** @return the {@link WorkerProfile} named `profileName`, when it exists */ @Override public Optional apply(String profileName) { return Optional.ofNullable(profileByName.get(profileName)); } + /** + * @return the {@link WorkerProfile} named `profileName` or throw {@link UnknownProfileException} when it does not exist + * @throws UnknownProfileException when `profileName` is unknown + */ public WorkerProfile getOrThrow(String profileName) { WorkerProfile profile = profileByName.get(profileName); if (profile != null) { @@ -64,12 +90,7 @@ public WorkerProfile getOrThrow(String profileName) { throw new UnknownProfileException(profileName); } - public void addProfile(WorkerProfile profile) { - if (profileByName.putIfAbsent(profile.getName(), profile) != null) { - throw new RuntimeException("profile '" + WorkforceProfiles.renderName(profile.getName()) + "' already exists!"); - } - } - + /** @return how many known profiles, including the baseline */ public int size() { return profileByName.size(); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java index 48860bca819..abc90983543 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java @@ -17,23 +17,34 @@ package org.apache.gobblin.temporal.dynamic; -import com.google.common.annotations.VisibleForTesting; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; +import javax.annotation.concurrent.ThreadSafe; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import lombok.Data; +/** + * Collection to map {@link WorkerProfile} names, each to a given set point. It might be "managed" by a {@link WorkforcePlan}, to reflect + * desired staffing, or else "unmanaged", where it might represent the current, actual per-worker scaling level. Those two might then be compared via + * {@link WorkforcePlan#calcStaffingDeltas(WorkforceStaffing)} to calculate necessary {@link StaffingDeltas} for attaining the "managed" workforce plan. + */ +@ThreadSafe public class WorkforceStaffing { public static final long INITIALIZATION_PROVENANCE_EPOCH_MILLIS = 0L; // CAUTION: sentinel value only for use with `StaffingDeltas.ProfileDelta` - NOT for use with `WorkforceStaffing::reviseStaffing`! public static final long UNKNOWN_PROVENANCE_EPOCH_MILLIS = -1L; + /** + * internal rep. for a set point, with associated provenance timestamp, that will be returned by {@link #calcDeltas(WorkforceStaffing, WorkforceProfiles)}, + * to inform debugging + */ @Data private static class SetPoint { private final int numWorkers; @@ -41,18 +52,21 @@ private static class SetPoint { } - private final Map setPointByName; + private final Map setPointsByName; + /** restricted-access ctor: instead use {@link #initialize(int)} */ private WorkforceStaffing() { - this.setPointByName = new ConcurrentHashMap<>(); + this.setPointsByName = new ConcurrentHashMap<>(); } + /** @return a new instance with `initialBaselineSetPoint` for the "baseline profile's" set point */ public static WorkforceStaffing initialize(int initialBaselineSetPoint) { WorkforceStaffing staffing = new WorkforceStaffing(); staffing.reviseStaffing(WorkforceProfiles.BASELINE_NAME, initialBaselineSetPoint, INITIALIZATION_PROVENANCE_EPOCH_MILLIS); return staffing; } + /** @return [for test init. brevity] a new instance with `initialBaselineSetPoint` for the "baseline profile" set point, plus multiple other set points */ @VisibleForTesting public static WorkforceStaffing initializeStaffing(int initialBaselineSetPoint, Map initialSetPointsByProfileName) { WorkforceStaffing staffing = initialize(initialBaselineSetPoint); @@ -62,37 +76,48 @@ public static WorkforceStaffing initializeStaffing(int initialBaselineSetPoint, return staffing; } + /** @return the staffing set point for the {@link WorkerProfile} named `profileName`, when it exists */ public Optional getStaffing(String profileName) { - return Optional.ofNullable(setPointByName.get(profileName)).map(SetPoint::getNumWorkers); + return Optional.ofNullable(setPointsByName.get(profileName)).map(SetPoint::getNumWorkers); } + /** update the staffing set point for the {@link WorkerProfile} named `profileName`, recording `provenanceEpochMillis` as the revision timestamp */ public void reviseStaffing(String profileName, int setPoint, long provenanceEpochMillis) { Preconditions.checkArgument(setPoint >= 0, "set points must be non-negative: '" + profileName + "' had " + setPoint); Preconditions.checkArgument(provenanceEpochMillis >= INITIALIZATION_PROVENANCE_EPOCH_MILLIS, "provenanceEpochMillis must be non-negative: '" + profileName + "' had " + provenanceEpochMillis); - setPointByName.put(profileName, new SetPoint(setPoint, provenanceEpochMillis)); + setPointsByName.put(profileName, new SetPoint(setPoint, provenanceEpochMillis)); + } + + /** update the staffing set point for the {@link WorkerProfile} named `profileName`, without recording any specific provenance timestamp */ + @VisibleForTesting + public void updateSetPoint(String profileName, int setPoint) { + reviseStaffing(profileName, setPoint, INITIALIZATION_PROVENANCE_EPOCH_MILLIS); } /** - * NOTE: so long as the same {@link WorkforcePlan} managed both this {@link WorkforceStaffing} and {@link WorkforceProfiles}, - * {@link WorkforceProfiles.UnknownProfileException} should NOT be possible. + * @return the {@link StaffingDeltas} between `this` as "the reference" versus `altStaffing`, using `profiles` to obtain {@link WorkerProfile}s. + * (A positive {@link StaffingDeltas.ProfileDelta#getDelta()} reflects an increase, while a negative, a decrease.) + * + * NOTE: when the same {@link WorkforcePlan} manages both this {@link WorkforceStaffing} and {@link WorkforceProfiles}, then + * {@link WorkforceProfiles.UnknownProfileException} should NEVER occur. */ - public synchronized StaffingDeltas calcDeltas(WorkforceStaffing reference, WorkforceProfiles profiles) { - Map frozenReferenceSetPointsByName = new HashMap<>(); // freeze entries for consistency amidst multiple traversals - reference.setPointByName.entrySet().forEach(entry -> frozenReferenceSetPointsByName.put(entry.getKey(), entry.getValue())); + public synchronized StaffingDeltas calcDeltas(WorkforceStaffing altStaffing, WorkforceProfiles profiles) { + Map frozenAltSetPointsByName = new HashMap<>(); // freeze entries for consistency amidst multiple traversals + altStaffing.setPointsByName.entrySet().forEach(entry -> frozenAltSetPointsByName.put(entry.getKey(), entry.getValue())); // not expecting any profile earlier in `reference` to no longer be set... (but defensive coding nonetheless) - List profileDeltas = frozenReferenceSetPointsByName.entrySet().stream() - .filter(entry -> !this.setPointByName.containsKey(entry.getKey())) + List profileDeltas = frozenAltSetPointsByName.entrySet().stream() + .filter(entry -> !this.setPointsByName.containsKey(entry.getKey())) .map(entry -> new StaffingDeltas.ProfileDelta(profiles.getOrThrow(entry.getKey()), 0 - entry.getValue().getNumWorkers(), UNKNOWN_PROVENANCE_EPOCH_MILLIS)) .collect(Collectors.toList()); - profileDeltas.addAll(this.setPointByName.entrySet().stream().map(entry -> { - Optional optEquivReferenceSetPoint = Optional.ofNullable(frozenReferenceSetPointsByName.get(entry.getKey())).map(SetPoint::getNumWorkers); + profileDeltas.addAll(this.setPointsByName.entrySet().stream().map(entry -> { + Optional optEquivAltSetPoint = Optional.ofNullable(frozenAltSetPointsByName.get(entry.getKey())).map(SetPoint::getNumWorkers); return new StaffingDeltas.ProfileDelta( profiles.getOrThrow(entry.getKey()), - entry.getValue().getNumWorkers() - optEquivReferenceSetPoint.orElse(0), + entry.getValue().getNumWorkers() - optEquivAltSetPoint.orElse(0), entry.getValue().getProvenanceEpochMillis()); } - ).filter(delta -> !delta.isUnchanged()) + ).filter(delta -> delta.isChange()) .collect(Collectors.toList())); return new StaffingDeltas(profileDeltas); } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSourceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSourceTest.java index fad35c94e5e..522cc491c55 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSourceTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSourceTest.java @@ -33,7 +33,9 @@ import org.apache.hadoop.fs.Path; import org.mockito.ArgumentCaptor; +import org.mockito.Mock; import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -42,22 +44,23 @@ import org.apache.gobblin.util.io.SeekableFSInputStream; +/** Test {@link FsScalingDirectiveSource} */ public class FsScalingDirectiveSourceTest { private static final String DIRECTIVES_DIR = "/test/dynamic/directives"; private static final String ERRORS_DIR = "/test/dynamic/errors"; - private FileSystem fileSystem; + @Mock private FileSystem fileSystem; private FsScalingDirectiveSource source; private static final ScalingDirectiveParser parser = new ScalingDirectiveParser(); @BeforeMethod public void setUp() { - fileSystem = Mockito.mock(FileSystem.class); + MockitoAnnotations.openMocks(this); source = new FsScalingDirectiveSource(fileSystem, DIRECTIVES_DIR, Optional.of(ERRORS_DIR)); } @Test - public void getScalingDirectivesWhenAllValidFiles() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + public void getScalingDirectivesWhenAllValidSyntax() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { String[] fileNames = { "1700010000.=4", "1700020000.new_profile=7,+(a.b.c=7,x.y=five)", @@ -77,7 +80,7 @@ public void getScalingDirectivesWhenAllValidFiles() throws IOException, ScalingD } @Test - public void getScalingDirectivesWhileIgnoringInvalidEntries() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + public void getScalingDirectivesWhileRejectingEachInvalidEntry() throws IOException { String[] fileNames = { "1700010000.=4", // still returned... although it would later be rejected as `WorkforcePlan.IllegalRevisionException.UnrecognizedProfile` @@ -86,7 +89,7 @@ public void getScalingDirectivesWhileIgnoringInvalidEntries() throws IOException // rejected: illegal syntax will fail to parse "completely invalid", "1700040000.another_profile=3,+(a.b.c=8,x.y=six)", - // rejected: because we later mock this as a dir, but a directive MUST be a file + // rejected: because we later mock this as a dir, while a directive MUST be a file "1700046000.acutally_a_dir=6,-(b.a,y.x)", "1700050000.new_profile=9", // rejected: because Removing must list key names, NOT key-value pairs @@ -100,11 +103,15 @@ public void getScalingDirectivesWhileIgnoringInvalidEntries() throws IOException List directives = source.getScalingDirectives(); Assert.assertEquals(directives.size(), 5); - Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); - Assert.assertEquals(directives.get(1), parseDirective(fileNames[1]), "fileNames[" + 1 + "] = " + fileNames[1]); - Assert.assertEquals(directives.get(2), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); - Assert.assertEquals(directives.get(3), parseDirective(fileNames[4]), "fileNames[" + 4 + "] = " + fileNames[4]); - Assert.assertEquals(directives.get(4), parseDirective(fileNames[6]), "fileNames[" + 6 + "] = " + fileNames[6]); + try { + Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); + Assert.assertEquals(directives.get(1), parseDirective(fileNames[1]), "fileNames[" + 1 + "] = " + fileNames[1]); + Assert.assertEquals(directives.get(2), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); + Assert.assertEquals(directives.get(3), parseDirective(fileNames[4]), "fileNames[" + 4 + "] = " + fileNames[4]); + Assert.assertEquals(directives.get(4), parseDirective(fileNames[6]), "fileNames[" + 6 + "] = " + fileNames[6]); + } catch (ScalingDirectiveParser.InvalidSyntaxException e) { + Assert.fail("Unexpected parsing error (with test directive)!", e); + } // lastly, verify `ERRORS_DIR` acknowledgements (i.e. FS object rename) work as expected: ArgumentCaptor sourcePathCaptor = ArgumentCaptor.forClass(Path.class); @@ -125,7 +132,7 @@ public void getScalingDirectivesWhileIgnoringInvalidEntries() throws IOException } @Test - public void getScalingDirectivesWhileIgnoringOutOfOrderEntries() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + public void getScalingDirectivesWhileRejectingOutOfOrderEntries() throws IOException { String[] fileNames = { "1700010000.=4", "1700030000.new_profile=7,+(a.b.c=7,x.y=five)", @@ -140,8 +147,12 @@ public void getScalingDirectivesWhileIgnoringOutOfOrderEntries() throws IOExcept List directives = source.getScalingDirectives(); Assert.assertEquals(directives.size(), 2); - Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); - Assert.assertEquals(directives.get(1), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); + try { + Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); + Assert.assertEquals(directives.get(1), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); + } catch (ScalingDirectiveParser.InvalidSyntaxException e) { + Assert.fail("Unexpected parsing error (with test directive)!", e); + } // lastly, verify `ERRORS_DIR` acknowledgements (i.e. FS object rename) work as expected: ArgumentCaptor sourcePathCaptor = ArgumentCaptor.forClass(Path.class); @@ -162,7 +173,7 @@ public void getScalingDirectivesWhileIgnoringOutOfOrderEntries() throws IOExcept } @Test - public void getScalingDirectivesWithOverlayPlaceholders() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + public void getScalingDirectivesWithOverlayPlaceholders() throws IOException { String[] fileNames = { "1700010000.=4", "1700020000.some_profile=9,+(...)", @@ -170,46 +181,56 @@ public void getScalingDirectivesWithOverlayPlaceholders() throws IOException, Sc "1700040000.some_profile=3", "1700050000.other_profile=10" }; - String addingOverlayDef = "a.b.c=7,x.y=five"; // for [1] - String removingOverlayDef = "b.c,y.z.a"; // for [2] FileStatus[] fileStatuses = Streams.mapWithIndex(Arrays.stream(fileNames), (fileName, i) -> createFileStatus(fileName, 1000 * (i + 1)) ).toArray(FileStatus[]::new); Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); + + String addingOverlayDef = "a.b.c=7,x.y=five"; // for [1] + String removingOverlayDef = "b.c,y.z.a"; // for [2] Mockito.when(fileSystem.open(new Path(DIRECTIVES_DIR, fileNames[1]))).thenReturn(createInputStreamFromString(addingOverlayDef)); Mockito.when(fileSystem.open(new Path(DIRECTIVES_DIR, fileNames[2]))).thenReturn(createInputStreamFromString(removingOverlayDef)); List directives = source.getScalingDirectives(); Assert.assertEquals(directives.size(), fileNames.length); - Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); - Assert.assertEquals(directives.get(1), parseDirective(fileNames[1].replace("...", addingOverlayDef)), "fileNames[" + 1 + "] = " + fileNames[1]); - Assert.assertEquals(directives.get(2), parseDirective(fileNames[2].replace("...", removingOverlayDef)), "fileNames[" + 2 + "] = " + fileNames[2]); - Assert.assertEquals(directives.get(3), parseDirective(fileNames[3]), "fileNames[" + 3 + "] = " + fileNames[3]); - Assert.assertEquals(directives.get(4), parseDirective(fileNames[4]), "fileNames[" + 4 + "] = " + fileNames[4]); + try { + Assert.assertEquals(directives.get(0), parseDirective(fileNames[0]), "fileNames[" + 0 + "] = " + fileNames[0]); + Assert.assertEquals(directives.get(1), parseDirective(fileNames[1].replace("...", addingOverlayDef)), "fileNames[" + 1 + "] = " + fileNames[1]); + Assert.assertEquals(directives.get(2), parseDirective(fileNames[2].replace("...", removingOverlayDef)), "fileNames[" + 2 + "] = " + fileNames[2]); + Assert.assertEquals(directives.get(3), parseDirective(fileNames[3]), "fileNames[" + 3 + "] = " + fileNames[3]); + Assert.assertEquals(directives.get(4), parseDirective(fileNames[4]), "fileNames[" + 4 + "] = " + fileNames[4]); + } catch (ScalingDirectiveParser.InvalidSyntaxException e) { + Assert.fail("Unexpected parsing error (with test directive)!", e); + } Mockito.verify(fileSystem, Mockito.never()).rename(Mockito.any(), Mockito.any()); } @Test - public void getScalingDirectivesWithOverlayPlaceholdersButInvalidDefinitions() throws IOException, ScalingDirectiveParser.InvalidSyntaxException { + public void getScalingDirectivesWithOverlayPlaceholdersButInvalidDefinitions() throws IOException { String[] fileNames = { "1700020000.some_profile=9,+(...)", "1700030000.other_profile=2,-(...)", "1700070000.=10" }; - // NOTE: switch these, so the overlay defs are invalid from `addingOverlayDef` with Removing and `removingOverlayDef` with Adding - String addingOverlayDef = "a.b.c=7,x.y=five"; // for [1] - String removingOverlayDef = "b.c,y.z.a"; // for [0] FileStatus[] fileStatuses = Streams.mapWithIndex(Arrays.stream(fileNames), (fileName, i) -> createFileStatus(fileName, 1000 * (i + 1)) ).toArray(FileStatus[]::new); Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); + + // NOTE: switch these, so the overlay defs are invalid: `addingOverlayDef` with Removing and `removingOverlayDef` with Adding + String addingOverlayDef = "a.b.c=7,x.y=five"; // for [1] + String removingOverlayDef = "b.c,y.z.a"; // for [0] Mockito.when(fileSystem.open(new Path(DIRECTIVES_DIR, fileNames[0]))).thenReturn(createInputStreamFromString(removingOverlayDef)); Mockito.when(fileSystem.open(new Path(DIRECTIVES_DIR, fileNames[1]))).thenReturn(createInputStreamFromString(addingOverlayDef)); List directives = source.getScalingDirectives(); Assert.assertEquals(directives.size(), 1); - Assert.assertEquals(directives.get(0), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); + try { + Assert.assertEquals(directives.get(0), parseDirective(fileNames[2]), "fileNames[" + 2 + "] = " + fileNames[2]); + } catch (ScalingDirectiveParser.InvalidSyntaxException e) { + Assert.fail("Unexpected parsing error (with test directive)!", e); + } // lastly, verify `ERRORS_DIR` acknowledgements (i.e. FS object rename) work as expected: ArgumentCaptor sourcePathCaptor = ArgumentCaptor.forClass(Path.class); @@ -230,14 +251,14 @@ public void getScalingDirectivesWithOverlayPlaceholdersButInvalidDefinitions() t } @Test - public void getScalingDirectivesWithNoFiles() throws IOException { + public void getScalingDirectivesWithNoFilesReturnsEmpty() throws IOException { FileStatus[] fileStatuses = {}; Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenReturn(fileStatuses); Assert.assertTrue(source.getScalingDirectives().isEmpty()); } @Test(expectedExceptions = IOException.class) - public void getScalingDirectivesWithIOException() throws IOException { + public void getScalingDirectivesWithIOExceptionPassesThrough() throws IOException { Mockito.when(fileSystem.listStatus(new Path(DIRECTIVES_DIR))).thenThrow(new IOException()); source.getScalingDirectives(); } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileDerivationTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileDerivationTest.java index c10636ed8a0..e953298c66f 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileDerivationTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileDerivationTest.java @@ -20,7 +20,6 @@ import java.util.Optional; import java.util.function.Function; -import com.google.common.collect.Lists; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; @@ -28,12 +27,13 @@ import org.testng.Assert; +/** Test {@link ProfileDerivation} */ public class ProfileDerivationTest { @Test - public void testFormulateConfigSuccess() throws ProfileDerivation.UnknownBasisException { + public void testFormulateConfigWithSuccessfulBasisResolution() throws ProfileDerivation.UnknownBasisException { String basisProfileName = "testProfile"; - ProfileOverlay overlay = new ProfileOverlay.Adding(Lists.newArrayList(new ProfileOverlay.KVPair("key1", "value1B"))); + ProfileOverlay overlay = new ProfileOverlay.Adding(new ProfileOverlay.KVPair("key1", "value1B")); ProfileDerivation profileDerivation = new ProfileDerivation(basisProfileName, overlay); Function> basisResolver = name -> { @@ -50,12 +50,13 @@ public void testFormulateConfigSuccess() throws ProfileDerivation.UnknownBasisEx Assert.assertEquals(resultConfig.getString("key2"), "value2"); } + @Test public void testFormulateConfigUnknownBasis() { String basisProfileName = "foo"; try { ProfileDerivation derivation = new ProfileDerivation(basisProfileName, null); derivation.formulateConfig(ignore -> Optional.empty()); - Assert.fail("Expected UnknownBasisException"); + Assert.fail("Expected instead: UnknownBasisException"); } catch (ProfileDerivation.UnknownBasisException ube) { Assert.assertEquals(ube.getName(), basisProfileName); } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileOverlayTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileOverlayTest.java index ab9c70ef216..bca2dee0acd 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileOverlayTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ProfileOverlayTest.java @@ -17,7 +17,6 @@ package org.apache.gobblin.temporal.dynamic; -import com.google.common.collect.Lists; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; @@ -25,13 +24,13 @@ import org.testng.Assert; +/** Test {@link ProfileOverlay} */ public class ProfileOverlayTest { @Test public void testAddingApplyOverlay() { Config config = ConfigFactory.parseString("key1=value1A, key4=value4"); - ProfileOverlay.Adding adding = new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("key1", "value1B"), new ProfileOverlay.KVPair("key2", "value2"))); + ProfileOverlay.Adding adding = new ProfileOverlay.Adding(new ProfileOverlay.KVPair("key1", "value1B"), new ProfileOverlay.KVPair("key2", "value2")); Config updatedConfig = adding.applyOverlay(config); Assert.assertEquals(updatedConfig.getString("key1"), "value1B"); Assert.assertEquals(updatedConfig.getString("key2"), "value2"); @@ -41,7 +40,7 @@ public void testAddingApplyOverlay() { @Test public void testRemovingApplyOverlay() { Config config = ConfigFactory.parseString("key1=value1, key2=value2"); - ProfileOverlay.Removing removing = new ProfileOverlay.Removing(Lists.newArrayList("key1")); + ProfileOverlay.Removing removing = new ProfileOverlay.Removing("key1"); Config updatedConfig = removing.applyOverlay(config); Assert.assertFalse(updatedConfig.hasPath("key1")); Assert.assertEquals(updatedConfig.getString("key2"), "value2"); @@ -50,9 +49,8 @@ public void testRemovingApplyOverlay() { @Test public void testComboApplyOverlay() { Config config = ConfigFactory.parseString("key1=value1, key2=value2, key3=value3"); - ProfileOverlay.Adding adding = new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("key4", "value4"), new ProfileOverlay.KVPair("key5", "value5"))); - ProfileOverlay.Removing removing = new ProfileOverlay.Removing(Lists.newArrayList("key2", "key4")); + ProfileOverlay.Adding adding = new ProfileOverlay.Adding(new ProfileOverlay.KVPair("key4", "value4"), new ProfileOverlay.KVPair("key5", "value5")); + ProfileOverlay.Removing removing = new ProfileOverlay.Removing("key2", "key4"); ProfileOverlay.Combo combo = ProfileOverlay.Combo.normalize(adding, removing); Config updatedConfig = combo.applyOverlay(config); Assert.assertEquals(updatedConfig.getString("key1"), "value1"); @@ -70,10 +68,8 @@ public void testComboApplyOverlay() { @Test public void testAddingOver() { - ProfileOverlay.Adding adding1 = new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("key1", "value1"), new ProfileOverlay.KVPair("key2", "value2A"))); - ProfileOverlay.Adding adding2 = new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("key2", "value2B"), new ProfileOverlay.KVPair("key3", "value3"))); + ProfileOverlay.Adding adding1 = new ProfileOverlay.Adding(new ProfileOverlay.KVPair("key1", "value1"), new ProfileOverlay.KVPair("key2", "value2A")); + ProfileOverlay.Adding adding2 = new ProfileOverlay.Adding(new ProfileOverlay.KVPair("key2", "value2B"), new ProfileOverlay.KVPair("key3", "value3")); ProfileOverlay result = adding1.over(adding2); Config config = result.applyOverlay(ConfigFactory.empty()); Assert.assertEquals(config.getString("key1"), "value1"); @@ -83,15 +79,14 @@ public void testAddingOver() { @Test public void testRemovingOver() { - ProfileOverlay.Removing removing1 = new ProfileOverlay.Removing(Lists.newArrayList("key1", "key2")); - ProfileOverlay.Removing removing2 = new ProfileOverlay.Removing(Lists.newArrayList("key2", "key3")); + ProfileOverlay.Removing removing1 = new ProfileOverlay.Removing("key1", "key2"); + ProfileOverlay.Removing removing2 = new ProfileOverlay.Removing("key2", "key3"); ProfileOverlay result = removing1.over(removing2); Assert.assertTrue(result instanceof ProfileOverlay.Removing); ProfileOverlay.Removing removingResult = (ProfileOverlay.Removing) result; Assert.assertEqualsNoOrder(removingResult.getRemovalKeys().toArray(), new String[]{"key1", "key2", "key3"}); - Config config = - result.applyOverlay(ConfigFactory.parseString("key1=value1, key2=value2, key3=value3, key4=value4")); + Config config = result.applyOverlay(ConfigFactory.parseString("key1=value1, key2=value2, key3=value3, key4=value4")); Assert.assertFalse(config.hasPath("key1")); Assert.assertFalse(config.hasPath("key2")); Assert.assertFalse(config.hasPath("key3")); diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java index cd0a08b7bf7..890b3a01308 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParserTest.java @@ -22,13 +22,12 @@ import java.util.function.BiFunction; import java.util.stream.IntStream; -import com.google.common.collect.Lists; - import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import org.testng.Assert; +/** Test {@link ScalingDirectiveParser} */ public class ScalingDirectiveParserTest { private final ScalingDirectiveParser parser = new ScalingDirectiveParser(); @@ -52,7 +51,7 @@ public void parseUnnamedBaselineProfile() throws ScalingDirectiveParser.InvalidS } @Test - public void parseBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { + public void parseBaselineProfilePseudoIdentifier() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728436828.baseline()=6"); Assert.assertEquals(sd, new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 6, 1728436828L, Optional.empty())); } @@ -66,22 +65,22 @@ public void parseAddingOverlayWithCommaSep() throws ScalingDirectiveParser.Inval Assert.assertTrue(sd.getOptDerivedFrom().isPresent()); ProfileDerivation derivation = sd.getOptDerivedFrom().get(); Assert.assertEquals(derivation.getBasisProfileName(), "bar"); - Assert.assertEquals(derivation.getOverlay(), new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m", "sixteen")))); + Assert.assertEquals(derivation.getOverlay(), + new ProfileOverlay.Adding(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m", "sixteen"))); } @Test public void parseAddingOverlayWithSemicolonSep() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728439223.new_profile=32;baz+( a.b.c=7 ; l.m.n.o=sixteen )"); - Assert.assertEquals(sd, new ScalingDirective("new_profile", 32, 1728439223L, "baz", new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m.n.o", "sixteen"))))); + Assert.assertEquals(sd, new ScalingDirective("new_profile", 32, 1728439223L, "baz", + new ProfileOverlay.Adding(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m.n.o", "sixteen")))); } @Test public void parseAddingOverlayWithCommaSepUrlEncoded() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728460832.new_profile=16,baa+(a.b.c=7,l.m=sixteen%2C%20again)"); - Assert.assertEquals(sd, new ScalingDirective("new_profile", 16, 1728460832L, "baa", new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m", "sixteen, again"))))); + Assert.assertEquals(sd, new ScalingDirective("new_profile", 16, 1728460832L, "baa", + new ProfileOverlay.Adding(new ProfileOverlay.KVPair("a.b.c", "7"), new ProfileOverlay.KVPair("l.m", "sixteen, again")))); } @Test @@ -93,63 +92,60 @@ public void parseRemovingOverlayWithCommaSep() throws ScalingDirectiveParser.Inv Assert.assertTrue(sd.getOptDerivedFrom().isPresent()); ProfileDerivation derivation = sd.getOptDerivedFrom().get(); Assert.assertEquals(derivation.getBasisProfileName(), "my_profile"); - Assert.assertEquals(derivation.getOverlay(), new ProfileOverlay.Removing(Lists.newArrayList("x", "y.z"))); + Assert.assertEquals(derivation.getOverlay(), new ProfileOverlay.Removing("x", "y.z")); } @Test public void parseRemovingOverlayWithSemicolonSep() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728436499.other_profile=9;my_profile-(x.y;z.z)"); Assert.assertEquals(sd, new ScalingDirective("other_profile", 9, 1728436499L, "my_profile", - new ProfileOverlay.Removing(Lists.newArrayList("x.y", "z.z")))); + new ProfileOverlay.Removing("x.y", "z.z"))); } @Test public void parseAddingOverlayWithWhitespace() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse(" 1728998877 . another = 999 ; wow + ( t.r = jump%20 ; cb.az = foo%20#%20111 ) "); - Assert.assertEquals(sd, new ScalingDirective("another", 999, 1728998877L, "wow", new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("t.r", "jump "), - new ProfileOverlay.KVPair("cb.az", "foo # 111"))))); + Assert.assertEquals(sd, new ScalingDirective("another", 999, 1728998877L, "wow", + new ProfileOverlay.Adding(new ProfileOverlay.KVPair("t.r", "jump "), new ProfileOverlay.KVPair("cb.az", "foo # 111")))); } @Test public void parseRemovingOverlayWithWhitespace() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse(" 1728334455 . also = 77 , really - ( t.r , cb.az ) "); Assert.assertEquals(sd, new ScalingDirective("also", 77, 1728334455L, "really", - new ProfileOverlay.Removing(Lists.newArrayList("t.r", "cb.az")))); + new ProfileOverlay.Removing("t.r", "cb.az"))); } @Test - public void parseAddingOverlayWithUnnamedBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { + public void parseAddingOverlayUponUnnamedBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728441200.plus_profile=16,+(q.r.s=four,l.m=16)"); Assert.assertEquals(sd, new ScalingDirective("plus_profile", 16, 1728441200L, WorkforceProfiles.BASELINE_NAME, - new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("q.r.s", "four"), new ProfileOverlay.KVPair("l.m", "16"))))); + new ProfileOverlay.Adding(new ProfileOverlay.KVPair("q.r.s", "four"), new ProfileOverlay.KVPair("l.m", "16")))); } @Test - public void parseAddingOverlayWithBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { + public void parseAddingOverlayUponBaselineProfilePseudoIdentifier() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728443640.plus_profile=16,baseline()+(q.r=five,l.m=12)"); Assert.assertEquals(sd, new ScalingDirective("plus_profile", 16, 1728443640L, WorkforceProfiles.BASELINE_NAME, - new ProfileOverlay.Adding( - Lists.newArrayList(new ProfileOverlay.KVPair("q.r", "five"), new ProfileOverlay.KVPair("l.m", "12"))))); + new ProfileOverlay.Adding(new ProfileOverlay.KVPair("q.r", "five"), new ProfileOverlay.KVPair("l.m", "12")))); } @Test - public void parseRemovingOverlayWithUnnamedBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { + public void parseRemovingOverlayUponUnnamedBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("1728448521.extra_profile=0,-(a.b, c.d)"); Assert.assertEquals(sd, new ScalingDirective("extra_profile", 0, 1728448521L, WorkforceProfiles.BASELINE_NAME, - new ProfileOverlay.Removing(Lists.newArrayList("a.b", "c.d")))); + new ProfileOverlay.Removing("a.b", "c.d"))); } @Test - public void parseRemovingOverlayWithBaselineProfile() throws ScalingDirectiveParser.InvalidSyntaxException { + public void parseRemovingOverlayUponBaselineProfilePseudoIdentifier() throws ScalingDirectiveParser.InvalidSyntaxException { ScalingDirective sd = parser.parse("4.extra_profile=9,baseline()-(a.b, c.d)"); Assert.assertEquals(sd, new ScalingDirective("extra_profile", 9, 4L, WorkforceProfiles.BASELINE_NAME, - new ProfileOverlay.Removing(Lists.newArrayList("a.b", "c.d")))); + new ProfileOverlay.Removing("a.b", "c.d"))); } @Test - public void parseProfileIdTooLong() throws ScalingDirectiveParser.InvalidSyntaxException { + public void parseProfileIdTooLongThrows() throws ScalingDirectiveParser.InvalidSyntaxException { BiFunction fmtRemovingOverlaySyntax = (profileId, basisProfileId) -> { return "1728449000." + profileId + "=99," + basisProfileId + "-(foo,bar,baz)"; }; @@ -209,8 +205,8 @@ public void parseValidDirectives(String directive) throws ScalingDirectiveParser Assert.assertNotNull(parser.parse(directive)); } - @DataProvider(name = "validDirectivesToRoundTrip") - public String[][] validDirectivesForRoundTrip() { + @DataProvider(name = "validDirectivesToRoundTripWithAsString") + public String[][] validDirectivesToRoundTripWithAsString() { return new String[][]{ { "2.some_profile=15" }, { "6.extra_profile=9,the_basis+(a.b=foo, c.d=bar)" }, @@ -225,9 +221,9 @@ public String[][] validDirectivesForRoundTrip() { @Test( expectedExceptions = {}, - dataProvider = "validDirectivesToRoundTrip" + dataProvider = "validDirectivesToRoundTripWithAsString" ) - public void roundTripAsStringAfterParse(String directive) throws ScalingDirectiveParser.InvalidSyntaxException { + public void roundTripAsStringFollowingSuccessfulParse(String directive) throws ScalingDirectiveParser.InvalidSyntaxException { Assert.assertEquals(ScalingDirectiveParser.asString(parser.parse(directive)), directive); } @@ -269,7 +265,7 @@ public String[][] overlayPlaceholderDirectivesWithCompletionDefAndEquivalent() { expectedExceptions = {}, dataProvider = "overlayPlaceholderDirectivesWithCompletionDefAndEquivalent" ) - public void verifyPlaceholderEquivalence(String directiveWithPlaceholder, String overlayDefinition, String equivDirective) + public void verifyOverlayPlaceholderEquivalence(String directiveWithPlaceholder, String overlayDefinition, String equivDirective) throws ScalingDirectiveParser.InvalidSyntaxException { try { parser.parse(directiveWithPlaceholder); @@ -360,7 +356,7 @@ public String[][] invalidDirectives() { expectedExceptions = ScalingDirectiveParser.InvalidSyntaxException.class, dataProvider = "invalidDirectives" ) - public void parseInvalidDirectives(String directive) throws ScalingDirectiveParser.InvalidSyntaxException { + public void parseInvalidDirectivesThrows(String directive) throws ScalingDirectiveParser.InvalidSyntaxException { parser.parse(directive); } @@ -386,7 +382,7 @@ public String[][] overlayPlaceholderDirectivesWithInvalidCompletionDef() { expectedExceptions = ScalingDirectiveParser.InvalidSyntaxException.class, dataProvider = "overlayPlaceholderDirectivesWithInvalidCompletionDef" ) - public void verifyPlaceholderDefIsInvalid(String directiveWithPlaceholder, String invalidOverlayDefinition) + public void parsePlaceholderDefWithInvalidPlaceholderThrows(String directiveWithPlaceholder, String invalidOverlayDefinition) throws ScalingDirectiveParser.InvalidSyntaxException { try { parser.parse(directiveWithPlaceholder); diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java index cdaa875f264..d57dec60214 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java @@ -30,6 +30,7 @@ import org.testng.Assert; +/** Test {@link WorkforcePlan} */ public class WorkforcePlanTest { private Config baselineConfig = ConfigFactory.parseString("key1=value1, key2=value2"); private final int initialBaselineSetPoint = 10; @@ -46,6 +47,7 @@ public void reviseWithValidReSetPoint() throws WorkforcePlan.IllegalRevisionExce plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 1, 20000L)); Assert.assertEquals(plan.getLastRevisionEpochMillis(), 20000L); Assert.assertEquals(plan.getNumProfiles(), 1); + Assert.assertEquals(plan.peepStaffing(WorkforceProfiles.BASELINE_NAME), Optional.of(1), WorkforceProfiles.BASELINE_NAME_RENDERING); } @Test @@ -59,53 +61,59 @@ public void reviseWithValidDerivation() throws WorkforcePlan.IllegalRevisionExce Assert.assertEquals(plan.getNumProfiles(), 2); Config expectedConfig = ConfigFactory.parseString("key1=new_value, key2=value2, key4=value4"); Assert.assertEquals(plan.peepProfile("new_profile").getConfig(), expectedConfig); + + Assert.assertEquals(plan.peepStaffing(WorkforceProfiles.BASELINE_NAME), Optional.of(initialBaselineSetPoint), WorkforceProfiles.BASELINE_NAME_RENDERING); + Assert.assertEquals(plan.peepStaffing("new_profile"), Optional.of(5), "new_profile"); } @Test - public void reviseWhenNewerIgnoresOutOfOrderDirectives() throws WorkforcePlan.IllegalRevisionException { + public void reviseWhenNewerRejectsOutOfOrderDirectivesAndContinues() { AtomicInteger numErrors = new AtomicInteger(0); Assert.assertEquals(plan.getLastRevisionEpochMillis(), WorkforceStaffing.INITIALIZATION_PROVENANCE_EPOCH_MILLIS); Assert.assertEquals(plan.getNumProfiles(), 1); plan.reviseWhenNewer(Lists.newArrayList( new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 2, 100L), new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 3, 500L), - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 4, 200L), - createNewProfileDirective("new_profile", 5, 400L, WorkforceProfiles.BASELINE_NAME), + // (1) error: `OutdatedDirective` + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 4, 250L), + // (2) error: `OutdatedDirective` + createNewProfileDirective("new_profile", 5, 450L, WorkforceProfiles.BASELINE_NAME), // NOTE: the second attempt at derivation is NOT judged a duplicate, as the outdated timestamp of first attempt (above) meant it was ignored! createNewProfileDirective("new_profile", 6, 600L, WorkforceProfiles.BASELINE_NAME), new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7, 800L), - new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8, 700L) + // (3) error: `OutdatedDirective` + new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8, 750L) ), failure -> numErrors.incrementAndGet()); Assert.assertEquals(plan.getLastRevisionEpochMillis(), 800L); Assert.assertEquals(plan.getNumProfiles(), 2); - Assert.assertEquals(numErrors.get(), 0); + Assert.assertEquals(numErrors.get(), 3); Assert.assertEquals(plan.peepStaffing(WorkforceProfiles.BASELINE_NAME), Optional.of(7), WorkforceProfiles.BASELINE_NAME_RENDERING); Assert.assertEquals(plan.peepStaffing("new_profile"), Optional.of(6), "new_profile"); } @Test - public void reviseWhenNewerSwallowsErrors() throws WorkforcePlan.IllegalRevisionException { + public void reviseWhenNewerRejectsErrorsAndContinues() { AtomicInteger numErrors = new AtomicInteger(0); plan.reviseWhenNewer(Lists.newArrayList( new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 1, 100L), // (1) error: `UnrecognizedProfile` - new ScalingDirective("unknown_profile", 2, 250L), + new ScalingDirective("UNKNOWN_PROFILE", 2, 250L), createNewProfileDirective("new_profile", 3, 200L, WorkforceProfiles.BASELINE_NAME), // (2) error: `Redefinition` createNewProfileDirective("new_profile", 4, 450L, WorkforceProfiles.BASELINE_NAME), new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 5, 300L), // (3) error: `UnknownBasis` - createNewProfileDirective("other_profile", 6, 550L, "never_defined"), + createNewProfileDirective("other_profile", 6, 550L, "NEVER_DEFINED"), new ScalingDirective("new_profile", 7, 400L), - // ignored: out-of-order timestamp (not an error... see: `reviseWhenNewerIgnoresOutOfOrderDirectives`) + // (4) error: `OutdatedDirective` new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 8, 350L), createNewProfileDirective("another", 9, 500L, "new_profile") ), failure -> numErrors.incrementAndGet()); Assert.assertEquals(plan.getLastRevisionEpochMillis(), 500L); Assert.assertEquals(plan.getNumProfiles(), 3); - Assert.assertEquals(numErrors.get(), 3); + Assert.assertEquals(numErrors.get(), 4); Assert.assertEquals(plan.peepStaffing(WorkforceProfiles.BASELINE_NAME), Optional.of(5), WorkforceProfiles.BASELINE_NAME_RENDERING); Assert.assertEquals(plan.peepStaffing("new_profile"), Optional.of(7), "new_profile"); Assert.assertEquals(plan.peepStaffing("another"), Optional.of(9), "another"); @@ -138,15 +146,15 @@ public void calcStaffingDeltas() throws WorkforcePlan.IllegalRevisionException { deltas.getPerProfileDeltas().forEach(delta -> { switch (delta.getProfile().getName()) { case WorkforceProfiles.BASELINE_NAME: - Assert.assertEquals(delta.getDelta(), -94); + Assert.assertEquals(delta.getDelta(), 6 - 100); Assert.assertEquals(delta.getSetPointProvenanceEpochMillis(), 50L); break; case "new_profile": - Assert.assertEquals(delta.getDelta(), 4); + Assert.assertEquals(delta.getDelta(), 5 - 1); Assert.assertEquals(delta.getSetPointProvenanceEpochMillis(), 40L); break; case "another": - Assert.assertEquals(delta.getDelta(), 4); + Assert.assertEquals(delta.getDelta(), 4 - 0); Assert.assertEquals(delta.getSetPointProvenanceEpochMillis(), 60L); break; case "other_profile": // NOTE: should NOT be present (since delta == 0)! @@ -164,7 +172,7 @@ public void reviseWithOutdatedDirective() throws WorkforcePlan.IllegalRevisionEx @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.UnrecognizedProfile.class) public void reviseWithUnrecognizedProfileDirective() throws WorkforcePlan.IllegalRevisionException { - plan.revise(new ScalingDirective("unknown_profile", 7, 10000L)); + plan.revise(new ScalingDirective("UNKNOWN_PROFILE", 7, 10000L)); } @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.Redefinition.class) @@ -175,13 +183,13 @@ public void reviseWithRedefinitionDirective() throws WorkforcePlan.IllegalRevisi @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.UnknownBasis.class) public void reviseWithUnknownBasisDirective() throws WorkforcePlan.IllegalRevisionException { - plan.revise(createNewProfileDirective("new_profile", 5, 10000L, "never_defined")); + plan.revise(createNewProfileDirective("new_profile", 5, 10000L, "NEVER_DEFINED")); } private static ScalingDirective createNewProfileDirective(String profileName, int setPoint, long epochMillis, String basisProfileName) { return new ScalingDirective(profileName, setPoint, epochMillis, Optional.of( - new ProfileDerivation(basisProfileName, new ProfileOverlay.Adding(Lists.newArrayList( + new ProfileDerivation(basisProfileName, new ProfileOverlay.Adding( new ProfileOverlay.KVPair("key1", "new_value"), - new ProfileOverlay.KVPair("key4", "value4")))))); + new ProfileOverlay.KVPair("key4", "value4"))))); } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffingTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffingTest.java index 03e02e3248c..baed5f27788 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffingTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffingTest.java @@ -33,6 +33,7 @@ import static org.mockito.ArgumentMatchers.anyString; +/** Test {@link WorkforceStaffing} */ public class WorkforceStaffingTest { @Mock private WorkforceProfiles profiles; @@ -55,9 +56,10 @@ public void initializeShouldSetInitialBaselineSetPoint() { public void reviseStaffingShouldUpdateSetPoint() { String profileName = "testProfile"; WorkforceStaffing staffing = WorkforceStaffing.initialize(0); - staffing.reviseStaffing(profileName, 10, 1000L); + staffing.reviseStaffing(profileName, 10, 5000L); Assert.assertEquals(staffing.getStaffing(profileName), Optional.of(10)); + // NOTE: verify that `provenanceEpochMillis` is merely assoc. metadata, w/ no requirement for monotonic increase staffing.reviseStaffing(profileName, 17, 2000L); Assert.assertEquals(staffing.getStaffing(profileName), Optional.of(17)); } @@ -72,8 +74,8 @@ public void calcDeltasShouldReturnCorrectDeltas() { currentStaffing.reviseStaffing(heldSteadyProfileName, 9, 2000L); WorkforceStaffing improvedStaffing = WorkforceStaffing.initialize(7); - improvedStaffing.reviseStaffing(newlyAddedProfileName, 10, 3000L); - improvedStaffing.reviseStaffing(heldSteadyProfileName, 9, 4000L); + improvedStaffing.updateSetPoint(newlyAddedProfileName, 10); + improvedStaffing.updateSetPoint(heldSteadyProfileName, 9); StaffingDeltas deltas = improvedStaffing.calcDeltas(currentStaffing, profiles); Assert.assertEquals(deltas.getPerProfileDeltas().size(), 3); @@ -83,7 +85,7 @@ public void calcDeltasShouldReturnCorrectDeltas() { ImmutableMap expectedDeltaByProfileName = ImmutableMap.of( WorkforceProfiles.BASELINE_NAME, 2, subsequentlyUnreferencedProfileName, -3, - // NOTE: NOT present (when delta == 0)! + // NOTE: NOT present (since delta == 0)! // heldSteadyProfileName, 0, newlyAddedProfileName, 10 ); From a31969ea76dec72c4e16ee8e538f0115cb234da0 Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Wed, 6 Nov 2024 12:18:15 -0800 Subject: [PATCH 08/11] Add `ScalingDirectiveParser` javadoc --- .../dynamic/ScalingDirectiveParser.java | 148 +++++++++++++----- 1 file changed, 112 insertions(+), 36 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java index 8719692203d..64b967780d7 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java @@ -26,48 +26,107 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; +import com.google.common.annotations.VisibleForTesting; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; /** - * parse {@link ScalingDirective}s with syntax of the form: - * TIMESTAMP '.' WORKER_NAME '=' SETPOINT [ ( ',' | ';' ) WORKER_NAME ( '+(' KV_PAIR (*SEP* KV_PAIR)* ')' | '-( KEY (*SEP* KEY* ')' ) ] - * where *SEP* is either ',' or ';' (whichever did follow SETPOINT) - * the first form with '+' is an "adding" (upsert) overlay, the second form with '-' is a removing overlay - * allows for URL-encoded values in the KV_PAIRs and whitespace around any token + * Parser for {@link ScalingDirective} syntax of the form: + * TIMESTAMP '.' PROFILE_NAME '=' SET_POINT [ ( ',' | ';' ) PROFILE_NAME ( '+(' KV_PAIR ( KV_PAIR)* ')' | '-( KEY ( KEY)* ')' ) ] + * where: + * only ( TIMESTAMP '.' PROFILE_NAME '=' SET_POINT ) are non-optional * - 1728435970.my_profile=24 - 1728436821.=24 - 1728436828.baseline()=24 - - 1728439210.new_profile=16,bar+(a.b.c=7,l.m=sixteen) - 1728439223.new_profile=16;bar+(a.b.c=7;l.m=sixteen) - 1728460832.new_profile=16,bar+(a.b.c=7,l.m=sixteen%2C%20again) - - 1728436436.other_profile=9,my_profile-(x,y.z) - 1728436499.other_profile=9;my_profile-(x;y.z) - - 1728441200.plus_profile=16,+(a.b.c=7,l.m=sixteen) - 1728443640.plus_profile=16,baseline()+(a.b.c=7,l.m=sixteen) - - 1728448521.extra_profile=9,-(a.b, c.d) - 1728449978.extra_profile=9,baseline()-(a.b, c.d) -*/ + * is either ',' or ';' (whichever did follow SET_POINT) + * + * TIMESTAMP is millis-since-epoch + * + * PROFILE_NAME is a simple [a-zA-Z0-9_]+ identifier familiar from many programming languages. The special name "baseline()" is reserved + * for the baseline profile, which may alternatively be spelled as the empty identifier (""). + * + * SET_POINT must be a non-negative integer ('0' indicates no instances desired). + * + * The first form introduced by '+' is an "adding" (upsert) overlay; the second form with '-' is a "removing" overlay. + * + * KV_PAIR (for "adding") is an '='-delimited (KEY '=' VALUE), where VALUE may use URL-encoding to escape characters. + * + * KEY (for "removing" and in the "adding" KV_PAIR) is a '.'-separated sequence of alphanumeric identifier segments, as in a {@link java.util.Properties} + * or {@link com.typesafe.config.Config} name. + * + * Whitespace may appear around any tokens, though not within a KEY or a VALUE. Comments are unsupported. + * + * As an alternative to inlining a lengthy adding or removing overlay definition, {@link #OVERLAY_DEFINITION_PLACEHOLDER} may stand in to indicate that + * the definition itself will be supplied separately. Supply it and {@link OverlayPlaceholderNeedsDefinition#retryParsingWithDefinition(String)}, upon + * the same UNCHECKED exception (originally thrown by {@link #parse(String)}). + * + * Given this syntax is specifically designed for directives to appear as an HDFS file name, we enforce a {@link #MAX_PROFILE_IDENTIFIER_LENGTH} (== 100), + * to fit within the HDFS path segment limit (== 255), and therein avoid: + * org.apache.hadoop.hdfs.protocol.FSLimitException$PathComponentTooLongException: \ + * The maximum path component name limit of ... in directory ... is exceeded: limit=255 length=256 + * the max identifier length of 100 is chosen as follows: + * - limit == 255 + * - current millis-precision epoch timestamp requires 10 chars, yet reserve 16 for future-proofing to nanos-precision + * - reserve 30 chars for future use in syntax evolution + * - 200 = 255 [limit] - 16 [digit timestamp] - 1 ['.'] - 1 ['='] - 1 [',' / ';'] - 6 ['+(...)' / '-(...)'] - 30 [reserved... for future] + * - since a max of two profile identifiers, neither may exceed (200 / 2 == 100) chars + * + * Examples: + * - simply update the set point for the profile, `my_profile` (already existing/defined): + * 1728435970.my_profile=24 + * + * - update the set point of the baseline profile (equiv. forms): + * 1728436821.=24 + * 1728436828.baseline()=24 + * + * - define a new profile, `new_profile`, with a set point of 16 by deriving via "adding" overlay from the existing profile, `bar` (equiv. forms): + * 1728439210.new_profile=16,bar+(a.b.c=7,l.m=sixteen) + * 1728439223.new_profile=16;bar+(a.b.c=7;l.m=sixteen) + * + * - similar derivation, but demonstrating URL-encoding (to include ',' and literal space in the value): + * 1728460832.new_profile=16,bar+(a.b.c=7,l.m=sixteen%2C%20again) + * + * - define a new profile, `other_profile`, with a set point of 9 by deriving via "removing" overlay from the existing profile, `my_profile` (equiv. forms): + * 1728436436.other_profile=9,my_profile-(x,y.z) + * 1728436499.other_profile=9;my_profile-(x;y.z) + * + * - define a new profile, `plus_profile`, with an initial set point, via "adding" overlay upon the baseline profile (equiv. forms): + * 1728441200.plus_profile=5,+(a.b.c=7,l.m=sixteen) + * 1728443640.plus_profile=5,baseline()+(a.b.c=7,l.m=sixteen) + * + * - define a new profile, `extra_profile`, with an initial set point, via "removing" overlay upon the baseline profile (equiv. forms): + * 1728448521.extra_profile=14,-(a.b, c.d) + * 1728449978.extra_profile=14,baseline()-(a.b, c.d) + * + * - define a new profile with an initial set point, using {@link #OVERLAY_DEFINITION_PLACEHOLDER} syntax instead of inlining the overlay definition: + * 1728539479.and_also=21,baaz-(...) + * 1728547230.this_too=19,quux+(...) + */ @Slf4j public class ScalingDirectiveParser { + + /** Announce a syntax error within {@link #getDirective()} */ public static class InvalidSyntaxException extends Exception { + @Getter private final String directive; + public InvalidSyntaxException(String directive, String desc) { super("error: " + desc + ", in ==>" + directive + "<=="); this.directive = directive; } } + /** + * Report that {@link #getDirective()} used {@link #OVERLAY_DEFINITION_PLACEHOLDER} in lieu of inlining an "adding" or "removing" overlay definition. + * + * When the overlay definition is later recovered, pass it to {@link #retryParsingWithDefinition(String)} to re-attempt the parse. + */ public static class OverlayPlaceholderNeedsDefinition extends RuntimeException { + @Getter private final String directive; private final String overlaySep; private final boolean isAdding; - // ATTENTION: explicitly managed, rather than making this a non-static inner class so `definePlaceholder` may be `static` for testing, while avoiding: + // ATTENTION: explicitly manage a reference to `parser`, despite it being the enclosing class instance, instead of making this a non-static inner class. + // That allows `definePlaceholder` to be `static`, for simpler testability, while dodging: // Static declarations in inner classes are not supported at language level '8' private final ScalingDirectiveParser parser; @@ -79,7 +138,12 @@ private OverlayPlaceholderNeedsDefinition(String directive, String overlaySep, b this.parser = enclosing; } - // doesn't allow recursive placeholding... + /** + * Pass the missing `overlayDefinition` and re-attempt parsing. This DOES NOT allow nested placeholding, so `overlayDefinition` may not + * itself be/contain {@link #OVERLAY_DEFINITION_PLACEHOLDER}. + * + * @return the parsed {@link ScalingDirective} or throw {@link InvalidSyntaxException} + */ public ScalingDirective retryParsingWithDefinition(String overlayDefinition) throws InvalidSyntaxException { try { return this.parser.parse(definePlaceholder(this.directive, this.overlaySep, this.isAdding, overlayDefinition)); @@ -88,6 +152,8 @@ public ScalingDirective retryParsingWithDefinition(String overlayDefinition) thr } } + /** encapsulate the intricacies of splicing `overlayDefinition` into `directiveWithPlaceholder`, after attending to the necessary URL-encoding */ + @VisibleForTesting protected static String definePlaceholder(String directiveWithPlaceholder, String overlaySep, boolean isAdding, String overlayDefinition) { // use care to selectively `urlEncode` parts (but NOT the entire string), to avoid disrupting syntactic chars, like [,;=] String urlEncodedOverlayDef = Arrays.stream(overlayDefinition.split("\\s*" + overlaySep + "\\s*", -1)) // (neg. limit to disallow trailing empty strings) @@ -100,21 +166,14 @@ protected static String definePlaceholder(String directiveWithPlaceholder, Strin } }).collect(Collectors.joining(overlaySep)); - // correct any double-encoding of '%', in case it arrived url-encoded + // undo any double-encoding of '%', in case `overlayDefinition` arrived URL-encoded return directiveWithPlaceholder.replace(OVERLAY_DEFINITION_PLACEHOLDER, urlEncodedOverlayDef.replace("%25", "%")); } } + // TODO: syntax to remove an attr while ALSO "adding" (so not simply setting to the empty string) - [proposal: alt. form for KV_PAIR ::= ( KEY '|=|' )] - // TODO: also support non-inline overlay definitions - "(...)" - // consider an additional trailing "|" (or "," / ";") syntax when the additional props are only needed post-launch - // since we're primarily designed for HDFS file names, in addition, a max identifier length (to fit within HDFS path segment limit == 255) - // org.apache.hadoop.hdfs.protocol.FSLimitException$PathComponentTooLongException: \ - // The maximum path component name limit of ... in directory ... is exceeded: limit=255 length=256 - // 200 = 255 [limit] - 16 [digit timestamp] - 1 ['.'] - 1 ['='] - 1 [',' / ';'] - 6 ['+(...)' / '-(...)'] - 30 [reserved... for future] - // current millis-precision epoch timestamp requires 10 chars, but we reserve 16 for future-proofing to nanos-precision - // hence, neither (of the max two) profile identifiers may exceed 100 chars. - // TODO: syntax to indicate removing an attr during an addition + // syntax as described in class-level javadoc: private static final String DIRECTIVE_REGEX = "(?x) (?s) ^ \\s* (\\d+) \\s* \\. \\s* (\\w* | baseline\\(\\)) \\s* = \\s* (\\d+) " + "(?: \\s* ([;,]) \\s* (\\w* | baseline\\(\\)) \\s* (?: (\\+ \\s* \\( \\s* ([^)]*?) \\s* \\) ) | (- \\s* \\( \\s* ([^)]*?) \\s* \\) ) ) )? \\s* $"; @@ -130,6 +189,13 @@ protected static String definePlaceholder(String directiveWithPlaceholder, Strin private static final String BASELINE_ID = "baseline()"; + /** + * Parse `directive` into a {@link ScalingDirective} or throw {@link InvalidSyntaxException} + * + * When an overlay definition was not inlined because {@link #OVERLAY_DEFINITION_PLACEHOLDER} was used instead, throw the UNCHECKED exception + * {@link OverlayPlaceholderNeedsDefinition}, which facilitates a subsequent attempt to supply the overlay definition and + * {@link OverlayPlaceholderNeedsDefinition#retryParsingWithDefinition(String)} (a form of the Proxy pattern). + */ public ScalingDirective parse(String directive) throws InvalidSyntaxException { Matcher parsed = directivePattern.matcher(directive); if (parsed.matches()) { @@ -147,7 +213,7 @@ public ScalingDirective parse(String directive) throws InvalidSyntaxException { if (additionsStr.equals(OVERLAY_DEFINITION_PLACEHOLDER)) { throw new OverlayPlaceholderNeedsDefinition(directive, overlayIntroSep, true, this); } else if (!additionsStr.equals("")) { - for (String addStr : additionsStr.split("\\s*" + overlayIntroSep + "\\s*", -1)) { // (negative limit to disallow trailing empty strings) + for (String addStr : additionsStr.split("\\s*" + overlayIntroSep + "\\s*", -1)) { // (`-1` limit to disallow trailing empty strings) Matcher keyValueParsed = keyValuePattern.matcher(addStr); if (keyValueParsed.matches()) { additions.add(new ProfileOverlay.KVPair(keyValueParsed.group(1), urlDecode(directive, keyValueParsed.group(2)))); @@ -163,7 +229,7 @@ public ScalingDirective parse(String directive) throws InvalidSyntaxException { if (removalsStr.equals(OVERLAY_DEFINITION_PLACEHOLDER)) { throw new OverlayPlaceholderNeedsDefinition(directive, overlayIntroSep, false, this); } else if (!removalsStr.equals("")) { - for (String removeStr : removalsStr.split("\\s*" + overlayIntroSep + "\\s*", -1)) { // (negative limit to disallow trailing empty strings) + for (String removeStr : removalsStr.split("\\s*" + overlayIntroSep + "\\s*", -1)) { // (`-1` limit to disallow trailing empty strings) Matcher keyParsed = keyPattern.matcher(removeStr); if (keyParsed.matches()) { removalKeys.add(keyParsed.group(1)); @@ -181,6 +247,13 @@ public ScalingDirective parse(String directive) throws InvalidSyntaxException { } } + /** + * @return `directive` as a pretty-printed string + * + * NOTE: regardless of its length or content, the result inlines the entire overlay def, with {@link #OVERLAY_DEFINITION_PLACEHOLDER} NEVER used + * + * @see #parse(String), the inverse operation (approximately - modulo {@link #OVERLAY_DEFINITION_PLACEHOLDER}, noted above) + */ public static String asString(ScalingDirective directive) { StringBuilder sb = new StringBuilder(); sb.append(directive.getTimestampEpochMillis()).append('.').append(directive.getProfileName()).append('=').append(directive.getSetPoint()); @@ -210,6 +283,7 @@ public static String asString(ScalingDirective directive) { return sb.toString(); } + /** handle special naming of {@link #BASELINE_ID} and enforce {@link #MAX_PROFILE_IDENTIFIER_LENGTH} */ private static String identifyProfileName(String profileId, String directive) throws InvalidSyntaxException { if (profileId.length() > MAX_PROFILE_IDENTIFIER_LENGTH) { throw new InvalidSyntaxException(directive, "profile ID exceeds length limit (of " + MAX_PROFILE_IDENTIFIER_LENGTH + "): '" + profileId + "'"); @@ -217,6 +291,7 @@ private static String identifyProfileName(String profileId, String directive) th return profileId.equals(BASELINE_ID) ? WorkforceProfiles.BASELINE_NAME : profileId; } + /** @return `s`, URL-decoded as UTF-8 or throw {@link InvalidSyntaxException} */ private static String urlDecode(String directive, String s) throws InvalidSyntaxException { try { return java.net.URLDecoder.decode(s, URL_ENCODING_CHARSET); @@ -225,6 +300,7 @@ private static String urlDecode(String directive, String s) throws InvalidSyntax } } + /** @return `s`, URL-encoded as UTF-8 and wrap any {@link java.io.UnsupportedEncodingException}--which SHOULD NEVER HAPPEN!--as an unchecked exception */ private static String urlEncode(String s) { try { return URLEncoder.encode(s, URL_ENCODING_CHARSET); From c0c4ae9e2f14d79822eb11451e353bb6e909b38b Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Wed, 6 Nov 2024 12:58:35 -0800 Subject: [PATCH 09/11] fixup - mainly javadoc --- .../temporal/dynamic/ScalingDirective.java | 4 +- .../dynamic/ScalingDirectiveParser.java | 16 ++++---- .../temporal/dynamic/WorkforcePlan.java | 37 ++++++++++--------- .../temporal/dynamic/WorkforceStaffing.java | 4 +- .../temporal/dynamic/WorkforcePlanTest.java | 14 +++---- 5 files changed, 39 insertions(+), 36 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java index a95bef7541a..8af9e95249a 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirective.java @@ -23,9 +23,9 @@ /** - * Core abstraction to model scaling adjustment, which originates at a given moment in time. A directive provides a set point for a given worker profile. + * Core abstraction to model scaling adjustment: a directive originates at a specific moment in time to provide a set point for a given worker profile. * The set point is the number of instances presently desired for that profile. When naming a heretofore unknown worker profile, the directive MUST also - * define that new profile through a {@link ProfileDerivation} that references a known profile. Known worker profiles MUST NOT be redefined. + * define that new profile through a {@link ProfileDerivation} referencing a known profile. Once defined, a worker profile MUST NOT be redefined. */ @Data @RequiredArgsConstructor diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java index 64b967780d7..da1ea3144c2 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java @@ -59,10 +59,10 @@ * the definition itself will be supplied separately. Supply it and {@link OverlayPlaceholderNeedsDefinition#retryParsingWithDefinition(String)}, upon * the same UNCHECKED exception (originally thrown by {@link #parse(String)}). * - * Given this syntax is specifically designed for directives to appear as an HDFS file name, we enforce a {@link #MAX_PROFILE_IDENTIFIER_LENGTH} (== 100), - * to fit within the HDFS path segment limit (== 255), and therein avoid: - * org.apache.hadoop.hdfs.protocol.FSLimitException$PathComponentTooLongException: \ - * The maximum path component name limit of ... in directory ... is exceeded: limit=255 length=256 + * Given this syntax is specifically designed for directives to appear as HDFS file names, we enforce a {@link #MAX_PROFILE_IDENTIFIER_LENGTH} (== 100), + * to ensure fit within the HDFS path segment limit (== 255), and therein avoid: + * org.apache.hadoop.hdfs.protocol.FSLimitException$PathComponentTooLongException: \ + * The maximum path component name limit of ... in directory ... is exceeded: limit=255 length=256 * the max identifier length of 100 is chosen as follows: * - limit == 255 * - current millis-precision epoch timestamp requires 10 chars, yet reserve 16 for future-proofing to nanos-precision @@ -71,7 +71,7 @@ * - since a max of two profile identifiers, neither may exceed (200 / 2 == 100) chars * * Examples: - * - simply update the set point for the profile, `my_profile` (already existing/defined): + * - simply update the set point for the (already existing/defined) profile, `my_profile`: * 1728435970.my_profile=24 * * - update the set point of the baseline profile (equiv. forms): @@ -82,7 +82,7 @@ * 1728439210.new_profile=16,bar+(a.b.c=7,l.m=sixteen) * 1728439223.new_profile=16;bar+(a.b.c=7;l.m=sixteen) * - * - similar derivation, but demonstrating URL-encoding (to include ',' and literal space in the value): + * - similar derivation, but demonstrating URL-encoding (to preserve ',' and literal space in the value): * 1728460832.new_profile=16,bar+(a.b.c=7,l.m=sixteen%2C%20again) * * - define a new profile, `other_profile`, with a set point of 9 by deriving via "removing" overlay from the existing profile, `my_profile` (equiv. forms): @@ -127,7 +127,7 @@ public static class OverlayPlaceholderNeedsDefinition extends RuntimeException { private final boolean isAdding; // ATTENTION: explicitly manage a reference to `parser`, despite it being the enclosing class instance, instead of making this a non-static inner class. // That allows `definePlaceholder` to be `static`, for simpler testability, while dodging: - // Static declarations in inner classes are not supported at language level '8' + // Static declarations in inner classes are not supported at language level '8' private final ScalingDirectiveParser parser; private OverlayPlaceholderNeedsDefinition(String directive, String overlaySep, boolean isAdding, ScalingDirectiveParser enclosing) { @@ -173,7 +173,7 @@ protected static String definePlaceholder(String directiveWithPlaceholder, Strin // TODO: syntax to remove an attr while ALSO "adding" (so not simply setting to the empty string) - [proposal: alt. form for KV_PAIR ::= ( KEY '|=|' )] - // syntax as described in class-level javadoc: + // syntax (described in class-level javadoc): private static final String DIRECTIVE_REGEX = "(?x) (?s) ^ \\s* (\\d+) \\s* \\. \\s* (\\w* | baseline\\(\\)) \\s* = \\s* (\\d+) " + "(?: \\s* ([;,]) \\s* (\\w* | baseline\\(\\)) \\s* (?: (\\+ \\s* \\( \\s* ([^)]*?) \\s* \\) ) | (- \\s* \\( \\s* ([^)]*?) \\s* \\) ) ) )? \\s* $"; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java index ec82ae0cd9d..6ce0c1b9dac 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java @@ -46,8 +46,8 @@ private IllegalRevisionException(ScalingDirective directive, String msg) { } /** Illegal revision: directive arrived out of {@link ScalingDirective#getTimestampEpochMillis()} order */ - public static class OutdatedDirective extends IllegalRevisionException { - protected OutdatedDirective(ScalingDirective directive, long lastRevisionEpochMillis) { + public static class OutOfOrderDirective extends IllegalRevisionException { + protected OutOfOrderDirective(ScalingDirective directive, long lastRevisionEpochMillis) { super(directive, "directive for profile '" + directive.renderName() + "' precedes last revision at " + lastRevisionEpochMillis + ": " + directive); } @@ -61,13 +61,6 @@ protected Redefinition(ScalingDirective directive, ProfileDerivation proposedDer } } - /** Illegal revision: set point for an unknown worker profile */ - public static class UnrecognizedProfile extends IllegalRevisionException { - protected UnrecognizedProfile(ScalingDirective directive) { - super(directive, "unrecognized profile reference '" + directive.renderName() + "': " + directive); - } - } - /** Illegal revision: worker profile evolution from an unknown basis profile */ public static class UnknownBasis extends IllegalRevisionException { protected UnknownBasis(ScalingDirective directive, ProfileDerivation.UnknownBasisException ube) { @@ -75,13 +68,20 @@ protected UnknownBasis(ScalingDirective directive, ProfileDerivation.UnknownBasi + WorkforceProfiles.renderName(ube.getName()) + "': " + directive); } } + + /** Illegal revision: set point for an unknown worker profile */ + public static class UnrecognizedProfile extends IllegalRevisionException { + protected UnrecognizedProfile(ScalingDirective directive) { + super(directive, "unrecognized profile reference '" + directive.renderName() + "': " + directive); + } + } } private final WorkforceProfiles profiles; private final WorkforceStaffing staffing; @Getter private volatile long lastRevisionEpochMillis; - /** create new plan with `baselineConfig` with `initialSetPoint` of the initial, baseline worker profile */ + /** create new plan with the initial, baseline worker profile using `baselineConfig` at `initialSetPoint` */ public WorkforcePlan(Config baselineConfig, int initialSetPoint) { this.profiles = WorkforceProfiles.withBaseline(baselineConfig); this.staffing = WorkforceStaffing.initialize(initialSetPoint); @@ -97,7 +97,7 @@ public int getNumProfiles() { public synchronized void revise(ScalingDirective directive) throws IllegalRevisionException { String name = directive.getProfileName(); if (this.lastRevisionEpochMillis >= directive.getTimestampEpochMillis()) { - throw new IllegalRevisionException.OutdatedDirective(directive, this.lastRevisionEpochMillis); + throw new IllegalRevisionException.OutOfOrderDirective(directive, this.lastRevisionEpochMillis); }; Optional optExistingProfile = profiles.apply(name); Optional optDerivation = directive.getOptDerivedFrom(); @@ -113,27 +113,30 @@ public synchronized void revise(ScalingDirective directive) throws IllegalRevisi throw new IllegalRevisionException.UnknownBasis(directive, ube); } } - // TODO - make idempotent, since any retry attempts after a failure between `addProfile` and `reviseStaffing` would henceforth fail with - // `IllegalRevisionException.Redefinition`, despite us wishing to adjust the set point now that the new profile has been defined... + // TODO - make idempotent, since any retry attempt following failure between `addProfile` and `reviseStaffing` would thereafter fail with + // `IllegalRevisionException.Redefinition`, despite us wishing to adjust the set point for that new profile just defined... // how to ensure the profile def is the same / unchanged? (e.g. compare full profile `Config` equality)? + // NOTE: the current outcome would be a profile defined in `WorkforceProfiles` with no set point in `WorkforceStaffing`. fortunately, + // that would NOT lead to `calcStaffingDeltas` throwing {@link WorkforceProfiles.UnknownProfileException}! + this.staffing.reviseStaffing(name, directive.getSetPoint(), directive.getTimestampEpochMillis()); this.lastRevisionEpochMillis = directive.getTimestampEpochMillis(); } } /** - * Performs atomic bulk revision while enforcing `directives` ordering by {@link ScalingDirective#getTimestampEpochMillis()} + * Performs atomic bulk revision while enforcing `directives` ordering in accord with {@link ScalingDirective#getTimestampEpochMillis()} * - * This version catches {@link IllegalRevisionException}s, logging a warning message for any before continuing to process subsequent directives. + * This version catches {@link IllegalRevisionException}, to log a warning message before continuing to process subsequent directives. */ public synchronized void reviseWhenNewer(List directives) { reviseWhenNewer(directives, ire -> { log.warn("Failure: ", ire); }); } /** - * Performs atomic bulk revision while enforcing `directives` ordering by {@link ScalingDirective#getTimestampEpochMillis()} + * Performs atomic bulk revision while enforcing `directives` ordering in accord with {@link ScalingDirective#getTimestampEpochMillis()} * - * This version catches {@link IllegalRevisionException}s, feeding any to `illegalRevisionHandler` before continuing to process subsequent directives. + * This version catches {@link IllegalRevisionException}, to call `illegalRevisionHandler` before continuing to process subsequent directives. */ public synchronized void reviseWhenNewer(List directives, Consumer illegalRevisionHandler) { directives.stream().sequential() diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java index abc90983543..23bb2b49a3c 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java @@ -33,7 +33,7 @@ /** * Collection to map {@link WorkerProfile} names, each to a given set point. It might be "managed" by a {@link WorkforcePlan}, to reflect * desired staffing, or else "unmanaged", where it might represent the current, actual per-worker scaling level. Those two might then be compared via - * {@link WorkforcePlan#calcStaffingDeltas(WorkforceStaffing)} to calculate necessary {@link StaffingDeltas} for attaining the "managed" workforce plan. + * {@link WorkforcePlan#calcStaffingDeltas(WorkforceStaffing)} to calculate {@link StaffingDeltas} against the "managed" workforce plan. */ @ThreadSafe public class WorkforceStaffing { @@ -96,7 +96,7 @@ public void updateSetPoint(String profileName, int setPoint) { } /** - * @return the {@link StaffingDeltas} between `this` as "the reference" versus `altStaffing`, using `profiles` to obtain {@link WorkerProfile}s. + * @return the {@link StaffingDeltas} between `this` (as "the reference") and `altStaffing`, by using `profiles` to obtain {@link WorkerProfile}s. * (A positive {@link StaffingDeltas.ProfileDelta#getDelta()} reflects an increase, while a negative, a decrease.) * * NOTE: when the same {@link WorkforcePlan} manages both this {@link WorkforceStaffing} and {@link WorkforceProfiles}, then diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java index d57dec60214..fc99bd9f94f 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/WorkforcePlanTest.java @@ -164,17 +164,12 @@ public void calcStaffingDeltas() throws WorkforcePlan.IllegalRevisionException { }); } - @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.OutdatedDirective.class) - public void reviseWithOutdatedDirective() throws WorkforcePlan.IllegalRevisionException { + @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.OutOfOrderDirective.class) + public void reviseWithOutOfOrderDirective() throws WorkforcePlan.IllegalRevisionException { plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 7, 30000L)); plan.revise(new ScalingDirective(WorkforceProfiles.BASELINE_NAME, 12, 8000L)); } - @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.UnrecognizedProfile.class) - public void reviseWithUnrecognizedProfileDirective() throws WorkforcePlan.IllegalRevisionException { - plan.revise(new ScalingDirective("UNKNOWN_PROFILE", 7, 10000L)); - } - @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.Redefinition.class) public void reviseWithRedefinitionDirective() throws WorkforcePlan.IllegalRevisionException { plan.revise(createNewProfileDirective("new_profile", 5, 10000L, WorkforceProfiles.BASELINE_NAME)); @@ -186,6 +181,11 @@ public void reviseWithUnknownBasisDirective() throws WorkforcePlan.IllegalRevisi plan.revise(createNewProfileDirective("new_profile", 5, 10000L, "NEVER_DEFINED")); } + @Test(expectedExceptions = WorkforcePlan.IllegalRevisionException.UnrecognizedProfile.class) + public void reviseWithUnrecognizedProfileDirective() throws WorkforcePlan.IllegalRevisionException { + plan.revise(new ScalingDirective("UNKNOWN_PROFILE", 7, 10000L)); + } + private static ScalingDirective createNewProfileDirective(String profileName, int setPoint, long epochMillis, String basisProfileName) { return new ScalingDirective(profileName, setPoint, epochMillis, Optional.of( new ProfileDerivation(basisProfileName, new ProfileOverlay.Adding( From e492bf6bcb9d380e325c4430af0c979ab20c8ffb Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Wed, 6 Nov 2024 19:02:54 -0800 Subject: [PATCH 10/11] reformatting `@Getter` --- .../apache/gobblin/temporal/dynamic/ProfileDerivation.java | 3 +-- .../gobblin/temporal/dynamic/ScalingDirectiveParser.java | 6 ++---- .../apache/gobblin/temporal/dynamic/WorkforceProfiles.java | 3 +-- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java index 5eafe8a4332..1001150df3e 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ProfileDerivation.java @@ -34,8 +34,7 @@ public class ProfileDerivation { /** Flags when the basis profile was NOT found */ public static class UnknownBasisException extends Exception { - @Getter - private final String name; + @Getter private final String name; public UnknownBasisException(String basisName) { super("named '" + WorkforceProfiles.renderName(basisName) + "'"); this.name = basisName; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java index da1ea3144c2..816719fc4c5 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java @@ -106,8 +106,7 @@ public class ScalingDirectiveParser { /** Announce a syntax error within {@link #getDirective()} */ public static class InvalidSyntaxException extends Exception { - @Getter - private final String directive; + @Getter private final String directive; public InvalidSyntaxException(String directive, String desc) { super("error: " + desc + ", in ==>" + directive + "<=="); @@ -121,8 +120,7 @@ public InvalidSyntaxException(String directive, String desc) { * When the overlay definition is later recovered, pass it to {@link #retryParsingWithDefinition(String)} to re-attempt the parse. */ public static class OverlayPlaceholderNeedsDefinition extends RuntimeException { - @Getter - private final String directive; + @Getter private final String directive; private final String overlaySep; private final boolean isAdding; // ATTENTION: explicitly manage a reference to `parser`, despite it being the enclosing class instance, instead of making this a non-static inner class. diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java index 5cd3bbab27c..61f627297ea 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java @@ -32,8 +32,7 @@ public class WorkforceProfiles implements Function Date: Fri, 8 Nov 2024 10:19:10 -0800 Subject: [PATCH 11/11] misc javadoc updates --- .../dynamic/FsScalingDirectiveSource.java | 10 ++++--- .../dynamic/ScalingDirectiveParser.java | 27 +++++++++++-------- .../temporal/dynamic/WorkforcePlan.java | 7 ++--- .../temporal/dynamic/WorkforceProfiles.java | 2 +- .../temporal/dynamic/WorkforceStaffing.java | 15 +++++++---- 5 files changed, 37 insertions(+), 24 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java index 9901c1e14ca..6725c58b6e3 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/FsScalingDirectiveSource.java @@ -38,8 +38,8 @@ * A {@link ScalingDirectiveSource} that reads {@link ScalingDirective}s from a {@link FileSystem} directory, where each directive is the name * of a single file inside the directory. Directives too long for one filename path component MUST use the * {@link ScalingDirectiveParser#OVERLAY_DEFINITION_PLACEHOLDER} syntax and write their {@link ProfileDerivation} overlay as the file's data/content. - * Within-length scaling directives are no-data, zero-length files. When backed by HDFS, reading such zero-length scaling directive files is a - * NameNode-only operation, while their metadata-only nature additionally conserves NN object count/quota. + * Within-length scaling directives are no-data, zero-length files. When backed by HDFS, reading such zero-length scaling directive filenames is a + * NameNode-only operation, with their metadata-only nature conserving NN object count/quota. */ @Slf4j public class FsScalingDirectiveSource implements ScalingDirectiveSource { @@ -61,18 +61,20 @@ public FsScalingDirectiveSource(FileSystem fileSystem, String directivesDirPath, * Ignore invalid directives, and, when `optErrorsDirPath` was provided to the ctor, acknowledge each by moving it to a separate "errors" directory. * Regardless, always swallow {@link ScalingDirectiveParser.InvalidSyntaxException}. * - * Like un-parseable directives, so too are out-of-order directives invalid. This prevents late/out-of-order insertion and/or edits to the directives + * Like un-parseable directives, also invalid are out-of-order directives. This blocks late/out-of-order insertion and/or edits to the directives * stream. Each directive contains its own {@link ScalingDirective#getTimestampEpochMillis()} stated in its filename. Later-modtime directives are * rejected when directive-timestamp-order does not match {@link FileStatus} modtime order. In the case of a modtime tie, the directive with the * alphabetically-later filename is rejected. * - * NOTE: This returns ALL known directives, even those already returned by a prior invocation. + * ATTENTION: This returns ALL known directives, even those already returned by a prior invocation. When the underlying directory is unchanged + * before the next invocation, the result will be equal elements in the same order. * * @throws IOException when unable to read the directory (or file data, in the case of an overlay definition placeholder) */ @Override public List getScalingDirectives() throws IOException { List> directiveWithFileStatus = new ArrayList<>(); + // TODO: add caching by dir modtime to avoid re-listing the same, unchanged contents, while also avoiding repetitive parsing // to begin, just parse w/o worrying about ordering... that comes next for (FileStatus fileStatus : fileSystem.listStatus(dirPath)) { if (!fileStatus.isFile()) { diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java index 816719fc4c5..fa00c5630a0 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/ScalingDirectiveParser.java @@ -35,27 +35,31 @@ * Parser for {@link ScalingDirective} syntax of the form: * TIMESTAMP '.' PROFILE_NAME '=' SET_POINT [ ( ',' | ';' ) PROFILE_NAME ( '+(' KV_PAIR ( KV_PAIR)* ')' | '-( KEY ( KEY)* ')' ) ] * where: - * only ( TIMESTAMP '.' PROFILE_NAME '=' SET_POINT ) are non-optional + * only ( TIMESTAMP '.' PROFILE_NAME '=' SET_POINT ) are non-optional. An optional trailing definition for that profile may name the + * "basis" profile to derive from through an "adding" or "removing" overlay. * - * is either ',' or ';' (whichever did follow SET_POINT) + * is either ',' or ';' (whichever did follow SET_POINT); choose which to minimize escaping (a KV_PAIR's VALUE, by URL-encoding). * - * TIMESTAMP is millis-since-epoch + * TIMESTAMP is millis-since-epoch. * * PROFILE_NAME is a simple [a-zA-Z0-9_]+ identifier familiar from many programming languages. The special name "baseline()" is reserved * for the baseline profile, which may alternatively be spelled as the empty identifier (""). * * SET_POINT must be a non-negative integer ('0' indicates no instances desired). * - * The first form introduced by '+' is an "adding" (upsert) overlay; the second form with '-' is a "removing" overlay. + * When an overlay is present, the form introduced by '+' is an "adding" (upsert) overlay and the form prefixed by '-' is a "removing" overlay. + * @see ProfileOverlay for {@link ProfileOverlay.Adding} and {@link ProfileOverlay.Removing} semantics. * * KV_PAIR (for "adding") is an '='-delimited (KEY '=' VALUE), where VALUE may use URL-encoding to escape characters. * - * KEY (for "removing" and in the "adding" KV_PAIR) is a '.'-separated sequence of alphanumeric identifier segments, as in a {@link java.util.Properties} + * KEY (for "removing"; also in the "adding" KV_PAIR) is a '.'-separated sequence of alphanumeric identifier segments, as in a {@link java.util.Properties} * or {@link com.typesafe.config.Config} name. * - * Whitespace may appear around any tokens, though not within a KEY or a VALUE. Comments are unsupported. + * Whitespace may appear around any tokens, though not within a KEY or a VALUE. * - * As an alternative to inlining a lengthy adding or removing overlay definition, {@link #OVERLAY_DEFINITION_PLACEHOLDER} may stand in to indicate that + * Comments are unsupported. + * + * As an alternative to inlining a lengthy "adding" or "removing" overlay definition, {@link #OVERLAY_DEFINITION_PLACEHOLDER} may stand in to indicate that * the definition itself will be supplied separately. Supply it and {@link OverlayPlaceholderNeedsDefinition#retryParsingWithDefinition(String)}, upon * the same UNCHECKED exception (originally thrown by {@link #parse(String)}). * @@ -82,8 +86,9 @@ * 1728439210.new_profile=16,bar+(a.b.c=7,l.m=sixteen) * 1728439223.new_profile=16;bar+(a.b.c=7;l.m=sixteen) * - * - similar derivation, but demonstrating URL-encoding (to preserve ',' and literal space in the value): + * - similar derivation, but demonstrating URL-encoding, to preserve ',' and/or literal space in the value (equiv. forms): * 1728460832.new_profile=16,bar+(a.b.c=7,l.m=sixteen%2C%20again) + * 1728460832.new_profile=16;bar+(a.b.c=7;l.m=sixteen,%20again) * * - define a new profile, `other_profile`, with a set point of 9 by deriving via "removing" overlay from the existing profile, `my_profile` (equiv. forms): * 1728436436.other_profile=9,my_profile-(x,y.z) @@ -150,7 +155,7 @@ public ScalingDirective retryParsingWithDefinition(String overlayDefinition) thr } } - /** encapsulate the intricacies of splicing `overlayDefinition` into `directiveWithPlaceholder`, after attending to the necessary URL-encoding */ + /** encapsulate the intricacies of splicing `overlayDefinition` into `directiveWithPlaceholder`, while performing the necessary URL-encoding */ @VisibleForTesting protected static String definePlaceholder(String directiveWithPlaceholder, String overlaySep, boolean isAdding, String overlayDefinition) { // use care to selectively `urlEncode` parts (but NOT the entire string), to avoid disrupting syntactic chars, like [,;=] @@ -190,7 +195,7 @@ protected static String definePlaceholder(String directiveWithPlaceholder, Strin /** * Parse `directive` into a {@link ScalingDirective} or throw {@link InvalidSyntaxException} * - * When an overlay definition was not inlined because {@link #OVERLAY_DEFINITION_PLACEHOLDER} was used instead, throw the UNCHECKED exception + * When an overlay definition is not inlined and {@link #OVERLAY_DEFINITION_PLACEHOLDER} is used instead, throw the UNCHECKED exception * {@link OverlayPlaceholderNeedsDefinition}, which facilitates a subsequent attempt to supply the overlay definition and * {@link OverlayPlaceholderNeedsDefinition#retryParsingWithDefinition(String)} (a form of the Proxy pattern). */ @@ -250,7 +255,7 @@ public ScalingDirective parse(String directive) throws InvalidSyntaxException { * * NOTE: regardless of its length or content, the result inlines the entire overlay def, with {@link #OVERLAY_DEFINITION_PLACEHOLDER} NEVER used * - * @see #parse(String), the inverse operation (approximately - modulo {@link #OVERLAY_DEFINITION_PLACEHOLDER}, noted above) + * @see #parse(String), the (approximate) inverse operation (modulo {@link #OVERLAY_DEFINITION_PLACEHOLDER}, noted above) */ public static String asString(ScalingDirective directive) { StringBuilder sb = new StringBuilder(); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java index 6ce0c1b9dac..dde55556442 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java @@ -114,10 +114,11 @@ public synchronized void revise(ScalingDirective directive) throws IllegalRevisi } } // TODO - make idempotent, since any retry attempt following failure between `addProfile` and `reviseStaffing` would thereafter fail with - // `IllegalRevisionException.Redefinition`, despite us wishing to adjust the set point for that new profile just defined... - // how to ensure the profile def is the same / unchanged? (e.g. compare full profile `Config` equality)? + // `IllegalRevisionException.Redefinition`, despite us wishing to adjust the set point for that new profile defined just before the failure. + // - how to ensure the profile def is the same / unchanged? (e.g. compare full profile `Config` equality)? // NOTE: the current outcome would be a profile defined in `WorkforceProfiles` with no set point in `WorkforceStaffing`. fortunately, - // that would NOT lead to `calcStaffingDeltas` throwing {@link WorkforceProfiles.UnknownProfileException}! + // that would NOT lead to `calcStaffingDeltas` throwing {@link WorkforceProfiles.UnknownProfileException}! The out-of-band (manual) + // workaround/repair would be revision by another, later directive that provides the set point for that profile (WITHOUT providing the definition) this.staffing.reviseStaffing(name, directive.getSetPoint(), directive.getTimestampEpochMillis()); this.lastRevisionEpochMillis = directive.getTimestampEpochMillis(); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java index 61f627297ea..da19c1c98dd 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceProfiles.java @@ -30,7 +30,7 @@ @ThreadSafe public class WorkforceProfiles implements Function> { - /** Indicates `profileName` NOT found */ + /** Indicates {@link #getProfileName()} NOT found */ public static class UnknownProfileException extends RuntimeException { @Getter private final String profileName; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java index 23bb2b49a3c..2503e922a77 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforceStaffing.java @@ -31,9 +31,14 @@ /** - * Collection to map {@link WorkerProfile} names, each to a given set point. It might be "managed" by a {@link WorkforcePlan}, to reflect - * desired staffing, or else "unmanaged", where it might represent the current, actual per-worker scaling level. Those two might then be compared via - * {@link WorkforcePlan#calcStaffingDeltas(WorkforceStaffing)} to calculate {@link StaffingDeltas} against the "managed" workforce plan. + * Collection to map {@link WorkerProfile} names, each to a given set point. + * + * An instance might be "managed" by a {@link WorkforcePlan}, to reflect desired staffing, or else "unmanaged", where it might represent the + * current, actual per-worker scaling level. Those two could be compared via {@link #calcDeltas(WorkforceStaffing, WorkforceProfiles)}, to + * calculate the {@link StaffingDeltas} between the two (i.e. between the staffing for the "managed" workforce plan of record vs. the independently + * maintained, "unmanaged" staffing levels). + * + * TIP: for encapsulation simplicity, invoke the "managed" form through {@link WorkforcePlan#calcStaffingDeltas(WorkforceStaffing)} */ @ThreadSafe public class WorkforceStaffing { @@ -42,8 +47,8 @@ public class WorkforceStaffing { public static final long UNKNOWN_PROVENANCE_EPOCH_MILLIS = -1L; /** - * internal rep. for a set point, with associated provenance timestamp, that will be returned by {@link #calcDeltas(WorkforceStaffing, WorkforceProfiles)}, - * to inform debugging + * internal rep. for a set point, with associated provenance timestamp, to inform debugging, when returned by + * {@link #calcDeltas(WorkforceStaffing, WorkforceProfiles)} */ @Data private static class SetPoint {