From 8b8bb5a579dc4fa9a7496b391f10ad34402c256f Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Fri, 18 Oct 2024 04:28:07 -0700 Subject: [PATCH 01/23] Define Gobblin-on-Temporal`WorkforcePlan` and dynamic `ScalingDirective`s with parser --- .../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 +++++++ 14 files changed, 1477 insertions(+) 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/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 8eb3d4efd6f62579c2b27bfb7e630e1519be2077 Mon Sep 17 00:00:00 2001 From: Kip Kohn Date: Mon, 21 Oct 2024 11:35:57 -0700 Subject: [PATCH 02/23] change package name from `dynscale` to `dynamic` and add a few comments --- .../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 ------- 14 files changed, 1477 deletions(-) delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java delete 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/dynscale/ProfileDerivation.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java deleted file mode 100644 index 0e192c2c784..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileDerivation.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index ed36ee9c16f..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ProfileOverlay.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index 77d9b483c9a..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirective.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index 4dbbc065b78..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParser.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index 018af44ff95..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/StaffingDeltas.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index df67eb47224..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkerProfile.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index 7c61eb65c13..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforcePlan.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index b76c33562e2..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceProfiles.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index f54c8035170..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffing.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index d87df961c54..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileDerivationTest.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index 125487cde03..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ProfileOverlayTest.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index 282d7a52987..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/ScalingDirectiveParserTest.java +++ /dev/null @@ -1,247 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index 838964aa4fc..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforcePlanTest.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 deleted file mode 100644 index e34673195c3..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynscale/WorkforceStaffingTest.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.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 a788a8f37119fe22a47d23100542b8319914d45b Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Wed, 13 Nov 2024 20:47:43 +0530 Subject: [PATCH 03/23] initial changes for yarn integration for dynamic scaling --- .../temporal/dynamic/WorkforcePlan.java | 2 +- .../gobblin/temporal/yarn/YarnService.java | 18 +++ .../YarnServiceDynamicScalingManager.java | 103 ++++++++++++++++++ 3 files changed, 122 insertions(+), 1 deletion(-) create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnServiceDynamicScalingManager.java 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 dde55556442..b5c1c4ab999 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 @@ -78,7 +78,7 @@ protected UnrecognizedProfile(ScalingDirective directive) { } private final WorkforceProfiles profiles; - private final WorkforceStaffing staffing; + @Getter private final WorkforceStaffing staffing; @Getter private volatile long lastRevisionEpochMillis; /** create new plan with the initial, baseline worker profile using `baselineConfig` at `initialSetPoint` */ diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index c8fbd047c5d..c537c1a47dc 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -109,6 +109,8 @@ import org.apache.gobblin.yarn.event.ContainerReleaseRequest; import org.apache.gobblin.yarn.event.ContainerShutdownRequest; import org.apache.gobblin.yarn.event.NewContainerRequest; +import org.apache.gobblin.temporal.dynamic.StaffingDeltas; +import org.apache.gobblin.temporal.dynamic.WorkerProfile; /** * This class is responsible for all Yarn-related stuffs including ApplicationMaster registration, @@ -445,6 +447,22 @@ public synchronized boolean requestTargetNumberOfContainers(int numContainers, S return true; } + public synchronized void requestNewContainers(StaffingDeltas deltas) { + deltas.getPerProfileDeltas().forEach(profileDelta -> { + if (profileDelta.getDelta() > 0) { + LOGGER.info("Requesting {} new containers for profile {}", profileDelta.getDelta(), profileDelta.getProfile()); + requestContainersForWorkerProfile(profileDelta.getProfile(), profileDelta.getDelta()); + } + // TODO: Decide how to handle negative deltas + }); + } + + private synchronized void requestContainersForWorkerProfile(WorkerProfile workerProfile, int numContainers) { + int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); + int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); + requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores)); + } + // Request initial containers with default resource and helix tag private void requestInitialContainers(int containersRequested) { requestTargetNumberOfContainers(containersRequested, Collections.EMPTY_SET); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnServiceDynamicScalingManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnServiceDynamicScalingManager.java new file mode 100644 index 00000000000..7e1dc52415f --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnServiceDynamicScalingManager.java @@ -0,0 +1,103 @@ +package org.apache.gobblin.temporal.yarn; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.fs.FileSystem; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.google.common.base.Optional; +import com.google.common.util.concurrent.AbstractIdleService; + +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.dynamic.FsScalingDirectiveSource; +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; +import org.apache.gobblin.temporal.dynamic.StaffingDeltas; +import org.apache.gobblin.temporal.dynamic.WorkforcePlan; +import org.apache.gobblin.temporal.dynamic.WorkforceStaffing; +import org.apache.gobblin.util.ConfigUtils; +import org.apache.gobblin.util.ExecutorsUtils; + + +@Slf4j +public class YarnServiceDynamicScalingManager extends AbstractIdleService { + + private final String DYNAMIC_SCALING_PREFIX = GobblinTemporalConfigurationKeys.PREFIX + "dynamic.scaling."; + private final String DYNAMIC_SCALING_DIRECTIVES_DIR = DYNAMIC_SCALING_PREFIX + "directives.dir"; + private final String DYNAMIC_SCALING_ERRORS_DIR = DYNAMIC_SCALING_PREFIX + "errors.dir"; + private final String DYNAMIC_SCALING_INITIAL_DELAY = DYNAMIC_SCALING_PREFIX + "initial.delay"; + private final int DEFAULT_DYNAMIC_SCALING_INITIAL_DELAY_SECS = 60; + private final String DYNAMIC_SCALING_POLLING_INTERVAL = DYNAMIC_SCALING_PREFIX + "polling.interval"; + private final int DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS = 60; + private final Config config; + private final YarnService yarnService; + private final ScheduledExecutorService dynamicScalingExecutor; + private final FileSystem fs; + + public YarnServiceDynamicScalingManager(GobblinTemporalApplicationMaster appMaster) { + this.config = appMaster.getConfig(); + this.yarnService = appMaster.get_yarnService(); + this.dynamicScalingExecutor = Executors.newSingleThreadScheduledExecutor( + ExecutorsUtils.newThreadFactory(Optional.of(log), Optional.of("DynamicScalingExecutor"))); + this.fs = appMaster.getFs(); + } + + @Override + protected void startUp() { + int scheduleInterval = ConfigUtils.getInt(this.config, DYNAMIC_SCALING_POLLING_INTERVAL, + DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS); + int initialDelay = ConfigUtils.getInt(this.config, DYNAMIC_SCALING_INITIAL_DELAY, + DEFAULT_DYNAMIC_SCALING_INITIAL_DELAY_SECS); + + Config baselineConfig = ConfigFactory.empty(); + // TODO : Add required configs like initial containers, memory, cores, etc.. + WorkforcePlan workforcePlan = new WorkforcePlan(baselineConfig, 0); + ScalingDirectiveSource scalingDirectiveSource = new FsScalingDirectiveSource( + this.fs, + this.config.getString(DYNAMIC_SCALING_DIRECTIVES_DIR), + java.util.Optional.of(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) + ); + + log.info("Starting the " + YarnServiceDynamicScalingManager.class.getSimpleName()); + log.info("Scheduling the dynamic scaling task with an interval of {} seconds", scheduleInterval); + + this.dynamicScalingExecutor.scheduleAtFixedRate( + new YarnDynamicScalingRunnable(this.yarnService, workforcePlan, scalingDirectiveSource), + initialDelay, scheduleInterval, TimeUnit.SECONDS + ); + } + + @Override + protected void shutDown() { + log.info("Stopping the " + YarnServiceDynamicScalingManager.class.getSimpleName()); + ExecutorsUtils.shutdownExecutorService(this.dynamicScalingExecutor, Optional.of(log)); + } + + @AllArgsConstructor + static class YarnDynamicScalingRunnable implements Runnable { + private final YarnService yarnService; + private final WorkforcePlan workforcePlan; + private final ScalingDirectiveSource scalingDirectiveSource; + + @Override + public void run() { + WorkforceStaffing workforceStaffing = workforcePlan.getStaffing(); + try { + List scalingDirectives = scalingDirectiveSource.getScalingDirectives(); + workforcePlan.reviseWhenNewer(scalingDirectives); + StaffingDeltas deltas = workforcePlan.calcStaffingDeltas(workforceStaffing); + this.yarnService.requestNewContainers(deltas); + } catch (IOException e) { + log.error("Failed to get scaling directives", e); + } + } + } +} From f1c859c0a27f12345bbbb628585bd9a6a843a03e Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Mon, 18 Nov 2024 14:51:30 +0530 Subject: [PATCH 04/23] added worker profile config to be used while starting container --- .../yarn/DynamicScalingYarnService.java | 68 ++++++++++++++++++ ... => DynamicScalingYarnServiceManager.java} | 40 ++++------- .../GobblinTemporalApplicationMaster.java | 2 +- .../gobblin/temporal/yarn/YarnService.java | 71 ++++++++++--------- 4 files changed, 122 insertions(+), 59 deletions(-) create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/{YarnServiceDynamicScalingManager.java => DynamicScalingYarnServiceManager.java} (65%) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java new file mode 100644 index 00000000000..89fbc23ddd5 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -0,0 +1,68 @@ +package org.apache.gobblin.temporal.yarn; + +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import com.google.common.base.Optional; +import com.google.common.eventbus.EventBus; +import com.typesafe.config.Config; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.StaffingDeltas; +import org.apache.gobblin.temporal.dynamic.WorkerProfile; +import org.apache.gobblin.temporal.dynamic.WorkforcePlan; +import org.apache.gobblin.temporal.dynamic.WorkforceStaffing; +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + +@Slf4j +public class DynamicScalingYarnService extends YarnService { + + private final WorkforceStaffing workforceStaffing; + private final WorkforcePlan workforcePlan; + private final AtomicLong allocationRequestIdGenerator = new AtomicLong(0L); + + public DynamicScalingYarnService(Config config, String applicationName, String applicationId, + YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { + super(config, applicationName, applicationId, yarnConfiguration, fs, eventBus); + + this.workforceStaffing = WorkforceStaffing.initialize(getInitialContainers()); + this.workforcePlan = new WorkforcePlan(getConfig(), getInitialContainers()); + // Putting baseline profile in the map for default allocation request id (0) + this.allocationRequestIdToWorkerProfile.put(allocationRequestIdGenerator.getAndIncrement(), this.baselineWorkerProfile); + } + + public synchronized void reviseWorkforcePlanAndRequestNewContainers(List scalingDirectives) { + this.workforcePlan.reviseWhenNewer(scalingDirectives); + StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.workforceStaffing); + requestNewContainersForStaffingDeltas(deltas); + // update our staffing after requesting new containers + scalingDirectives.forEach(directive -> this.workforceStaffing.reviseStaffing( + directive.getProfileName(), directive.getSetPoint(), directive.getTimestampEpochMillis()) + ); + } + + private synchronized void requestNewContainersForStaffingDeltas(StaffingDeltas deltas) { + deltas.getPerProfileDeltas().forEach(profileDelta -> { + if (profileDelta.getDelta() > 0) { + log.info("Requesting {} new containers for profile {}", profileDelta.getDelta(), profileDelta.getProfile()); + requestContainersForWorkerProfile(profileDelta.getProfile(), profileDelta.getDelta()); + } + // TODO: Decide how to handle negative deltas + }); + } + + private synchronized void requestContainersForWorkerProfile(WorkerProfile workerProfile, int numContainers) { + int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); + int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); + long allocationRequestId = allocationRequestIdGenerator.getAndIncrement(); + this.allocationRequestIdToWorkerProfile.put(allocationRequestId, workerProfile); + requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores), Optional.of(allocationRequestId)); + } + +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnServiceDynamicScalingManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java similarity index 65% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnServiceDynamicScalingManager.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java index 7e1dc52415f..4e98dcf8c52 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnServiceDynamicScalingManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java @@ -2,6 +2,7 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -9,8 +10,6 @@ import org.apache.hadoop.fs.FileSystem; import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.google.common.base.Optional; import com.google.common.util.concurrent.AbstractIdleService; import lombok.AllArgsConstructor; @@ -20,15 +19,12 @@ import org.apache.gobblin.temporal.dynamic.FsScalingDirectiveSource; import org.apache.gobblin.temporal.dynamic.ScalingDirective; import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; -import org.apache.gobblin.temporal.dynamic.StaffingDeltas; -import org.apache.gobblin.temporal.dynamic.WorkforcePlan; -import org.apache.gobblin.temporal.dynamic.WorkforceStaffing; import org.apache.gobblin.util.ConfigUtils; import org.apache.gobblin.util.ExecutorsUtils; @Slf4j -public class YarnServiceDynamicScalingManager extends AbstractIdleService { +public class DynamicScalingYarnServiceManager extends AbstractIdleService { private final String DYNAMIC_SCALING_PREFIX = GobblinTemporalConfigurationKeys.PREFIX + "dynamic.scaling."; private final String DYNAMIC_SCALING_DIRECTIVES_DIR = DYNAMIC_SCALING_PREFIX + "directives.dir"; @@ -38,15 +34,16 @@ public class YarnServiceDynamicScalingManager extends AbstractIdleService { private final String DYNAMIC_SCALING_POLLING_INTERVAL = DYNAMIC_SCALING_PREFIX + "polling.interval"; private final int DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS = 60; private final Config config; - private final YarnService yarnService; + DynamicScalingYarnService dynamicScalingYarnService; private final ScheduledExecutorService dynamicScalingExecutor; private final FileSystem fs; - public YarnServiceDynamicScalingManager(GobblinTemporalApplicationMaster appMaster) { + public DynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { this.config = appMaster.getConfig(); - this.yarnService = appMaster.get_yarnService(); + this.dynamicScalingYarnService = (DynamicScalingYarnService) appMaster.get_yarnService(); this.dynamicScalingExecutor = Executors.newSingleThreadScheduledExecutor( - ExecutorsUtils.newThreadFactory(Optional.of(log), Optional.of("DynamicScalingExecutor"))); + ExecutorsUtils.newThreadFactory(com.google.common.base.Optional.of(log), + com.google.common.base.Optional.of("DynamicScalingExecutor"))); this.fs = appMaster.getFs(); } @@ -57,44 +54,37 @@ protected void startUp() { int initialDelay = ConfigUtils.getInt(this.config, DYNAMIC_SCALING_INITIAL_DELAY, DEFAULT_DYNAMIC_SCALING_INITIAL_DELAY_SECS); - Config baselineConfig = ConfigFactory.empty(); - // TODO : Add required configs like initial containers, memory, cores, etc.. - WorkforcePlan workforcePlan = new WorkforcePlan(baselineConfig, 0); ScalingDirectiveSource scalingDirectiveSource = new FsScalingDirectiveSource( this.fs, this.config.getString(DYNAMIC_SCALING_DIRECTIVES_DIR), - java.util.Optional.of(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) + Optional.ofNullable(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) ); - log.info("Starting the " + YarnServiceDynamicScalingManager.class.getSimpleName()); + log.info("Starting the " + DynamicScalingYarnServiceManager.class.getSimpleName()); log.info("Scheduling the dynamic scaling task with an interval of {} seconds", scheduleInterval); this.dynamicScalingExecutor.scheduleAtFixedRate( - new YarnDynamicScalingRunnable(this.yarnService, workforcePlan, scalingDirectiveSource), + new DynamicScalingYarnServiceRunnable(this.dynamicScalingYarnService, scalingDirectiveSource), initialDelay, scheduleInterval, TimeUnit.SECONDS ); } @Override protected void shutDown() { - log.info("Stopping the " + YarnServiceDynamicScalingManager.class.getSimpleName()); - ExecutorsUtils.shutdownExecutorService(this.dynamicScalingExecutor, Optional.of(log)); + log.info("Stopping the " + DynamicScalingYarnServiceManager.class.getSimpleName()); + ExecutorsUtils.shutdownExecutorService(this.dynamicScalingExecutor, com.google.common.base.Optional.of(log)); } @AllArgsConstructor - static class YarnDynamicScalingRunnable implements Runnable { - private final YarnService yarnService; - private final WorkforcePlan workforcePlan; + static class DynamicScalingYarnServiceRunnable implements Runnable { + private final DynamicScalingYarnService dynamicScalingYarnService; private final ScalingDirectiveSource scalingDirectiveSource; @Override public void run() { - WorkforceStaffing workforceStaffing = workforcePlan.getStaffing(); try { List scalingDirectives = scalingDirectiveSource.getScalingDirectives(); - workforcePlan.reviseWhenNewer(scalingDirectives); - StaffingDeltas deltas = workforcePlan.calcStaffingDeltas(workforceStaffing); - this.yarnService.requestNewContainers(deltas); + dynamicScalingYarnService.reviseWorkforcePlanAndRequestNewContainers(scalingDirectives); } catch (IOException e) { log.error("Failed to get scaling directives", e); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/GobblinTemporalApplicationMaster.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/GobblinTemporalApplicationMaster.java index b7957bd9a26..3efadb11b38 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/GobblinTemporalApplicationMaster.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/GobblinTemporalApplicationMaster.java @@ -114,7 +114,7 @@ public GobblinTemporalApplicationMaster(String applicationName, String applicati protected YarnService buildTemporalYarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, FileSystem fs) throws Exception { - return new YarnService(config, applicationName, applicationId, yarnConfiguration, fs, this.eventBus); + return new DynamicScalingYarnService(config, applicationName, applicationId, yarnConfiguration, fs, this.eventBus); } /** diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index c537c1a47dc..ae83402d12b 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -36,7 +36,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.IntStream; -import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -109,8 +108,8 @@ import org.apache.gobblin.yarn.event.ContainerReleaseRequest; import org.apache.gobblin.yarn.event.ContainerShutdownRequest; import org.apache.gobblin.yarn.event.NewContainerRequest; -import org.apache.gobblin.temporal.dynamic.StaffingDeltas; import org.apache.gobblin.temporal.dynamic.WorkerProfile; +import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; /** * This class is responsible for all Yarn-related stuffs including ApplicationMaster registration, @@ -133,6 +132,7 @@ class YarnService extends AbstractIdleService { //Default helix instance tag derived from cluster level config private final String helixInstanceTags; + @Getter(AccessLevel.PROTECTED) private final Config config; private final EventBus eventBus; @@ -149,6 +149,7 @@ class YarnService extends AbstractIdleService { private final NMClientAsync nmClientAsync; private final ExecutorService containerLaunchExecutor; + @Getter(AccessLevel.PROTECTED) private final int initialContainers; private final int requestedContainerMemoryMbs; private final int requestedContainerCores; @@ -160,7 +161,6 @@ class YarnService extends AbstractIdleService { private final Optional containerJvmArgs; private final String containerTimezone; - private final String proxyJvmArgs; @Getter(AccessLevel.PROTECTED) private volatile Optional maxResourceCapacity = Optional.absent(); @@ -202,6 +202,8 @@ class YarnService extends AbstractIdleService { private volatile boolean shutdownInProgress = false; private final boolean jarCacheEnabled; + protected final WorkerProfile baselineWorkerProfile; + protected final ConcurrentMap allocationRequestIdToWorkerProfile = new ConcurrentHashMap<>(); public YarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { @@ -244,9 +246,6 @@ public YarnService(Config config, String applicationName, String applicationId, Optional.of(config.getString(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY)) : Optional.absent(); - this.proxyJvmArgs = config.hasPath(GobblinYarnConfigurationKeys.YARN_APPLICATION_PROXY_JVM_ARGS) ? - config.getString(GobblinYarnConfigurationKeys.YARN_APPLICATION_PROXY_JVM_ARGS) : StringUtils.EMPTY; - int numContainerLaunchThreads = ConfigUtils.getInt(config, GobblinYarnConfigurationKeys.MAX_CONTAINER_LAUNCH_THREADS_KEY, GobblinYarnConfigurationKeys.DEFAULT_MAX_CONTAINER_LAUNCH_THREADS); @@ -280,6 +279,9 @@ public YarnService(Config config, String applicationName, String applicationId, this.containerTimezone = ConfigUtils.getString(this.config, GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_TIMEZONE, GobblinYarnConfigurationKeys.DEFAULT_GOBBLIN_YARN_CONTAINER_TIMEZONE); this.jarCacheEnabled = ConfigUtils.getBoolean(this.config, GobblinYarnConfigurationKeys.JAR_CACHE_ENABLED, GobblinYarnConfigurationKeys.JAR_CACHE_ENABLED_DEFAULT); + + // Initialising this baseline worker profile to use as default worker profile in case allocation request id is not in map + this.baselineWorkerProfile = new WorkerProfile(WorkforceProfiles.BASELINE_NAME, this.config); } @SuppressWarnings("unused") @@ -442,27 +444,11 @@ public synchronized boolean requestTargetNumberOfContainers(int numContainers, S LOGGER.info("Trying to set numTargetContainers={}, in-use helix instances count is {}, container map size is {}", numContainers, inUseInstances.size(), this.containerMap.size()); - requestContainers(numContainers, Resource.newInstance(defaultContainerMemoryMbs, defaultContainerCores)); + requestContainers(numContainers, Resource.newInstance(defaultContainerMemoryMbs, defaultContainerCores), Optional.absent()); LOGGER.info("Current tag-container desired count:{}, tag-container allocated: {}", numContainers, this.allocatedContainerCountMap); return true; } - public synchronized void requestNewContainers(StaffingDeltas deltas) { - deltas.getPerProfileDeltas().forEach(profileDelta -> { - if (profileDelta.getDelta() > 0) { - LOGGER.info("Requesting {} new containers for profile {}", profileDelta.getDelta(), profileDelta.getProfile()); - requestContainersForWorkerProfile(profileDelta.getProfile(), profileDelta.getDelta()); - } - // TODO: Decide how to handle negative deltas - }); - } - - private synchronized void requestContainersForWorkerProfile(WorkerProfile workerProfile, int numContainers) { - int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); - int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); - requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores)); - } - // Request initial containers with default resource and helix tag private void requestInitialContainers(int containersRequested) { requestTargetNumberOfContainers(containersRequested, Collections.EMPTY_SET); @@ -471,7 +457,7 @@ private void requestInitialContainers(int containersRequested) { private void requestContainer(Optional preferredNode, Optional resourceOptional) { Resource desiredResource = resourceOptional.or(Resource.newInstance( this.requestedContainerMemoryMbs, this.requestedContainerCores)); - requestContainer(preferredNode, desiredResource); + requestContainer(preferredNode, desiredResource, Optional.absent()); } /** @@ -480,14 +466,14 @@ private void requestContainer(Optional preferredNode, Optional * @param numContainers * @param resource */ - private void requestContainers(int numContainers, Resource resource) { - LOGGER.info("Requesting {} containers with resource={}", numContainers, resource); + protected void requestContainers(int numContainers, Resource resource, Optional allocationRequestId) { + LOGGER.info("Requesting {} containers with resource={} and allocation request id = {}", numContainers, resource, allocationRequestId); IntStream.range(0, numContainers) - .forEach(i -> requestContainer(Optional.absent(), resource)); + .forEach(i -> requestContainer(Optional.absent(), resource, allocationRequestId)); } // Request containers with specific resource requirement - private void requestContainer(Optional preferredNode, Resource resource) { + private void requestContainer(Optional preferredNode, Resource resource, Optional allocationRequestId) { // Fail if Yarn cannot meet container resource requirements Preconditions.checkArgument(resource.getMemory() <= this.maxResourceCapacity.get().getMemory() && resource.getVirtualCores() <= this.maxResourceCapacity.get().getVirtualCores(), @@ -503,8 +489,11 @@ private void requestContainer(Optional preferredNode, Resource resource) priority.setPriority(priorityNum); String[] preferredNodes = preferredNode.isPresent() ? new String[] {preferredNode.get()} : null; + + long allocationRequestID = allocationRequestId.isPresent() ? allocationRequestId.get() : 0L; + this.amrmClientAsync.addContainerRequest( - new AMRMClient.ContainerRequest(resource, preferredNodes, null, priority)); + new AMRMClient.ContainerRequest(resource, preferredNodes, null, priority, allocationRequestID)); } protected ContainerLaunchContext newContainerLaunchContext(ContainerInfo containerInfo) @@ -608,16 +597,32 @@ protected ByteBuffer getSecurityTokens() throws IOException { @VisibleForTesting protected String buildContainerCommand(Container container, String helixParticipantId, String helixInstanceTag) { + long allocationRequestID = container.getAllocationRequestId(); + WorkerProfile workerProfile = this.allocationRequestIdToWorkerProfile.getOrDefault(allocationRequestID, + this.baselineWorkerProfile); + Config workerProfileConfig = workerProfile.getConfig(); + + double workerJvmMemoryXmxRatio = ConfigUtils.getDouble(workerProfileConfig, + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, + GobblinYarnConfigurationKeys.DEFAULT_CONTAINER_JVM_MEMORY_XMX_RATIO); + + int workerJvmMemoryOverheadMbs = ConfigUtils.getInt(workerProfileConfig, + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, + GobblinYarnConfigurationKeys.DEFAULT_CONTAINER_JVM_MEMORY_OVERHEAD_MBS); + + Optional workerJvmArgs = workerProfileConfig.hasPath(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY) ? + Optional.of(workerProfileConfig.getString(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY)) : + Optional.absent(); + String containerProcessName = GobblinTemporalYarnTaskRunner.class.getSimpleName(); StringBuilder containerCommand = new StringBuilder() .append(ApplicationConstants.Environment.JAVA_HOME.$()).append("/bin/java") - .append(" -Xmx").append((int) (container.getResource().getMemory() * this.jvmMemoryXmxRatio) - - this.jvmMemoryOverheadMbs).append("M") + .append(" -Xmx").append((int) (container.getResource().getMemory() * workerJvmMemoryXmxRatio) - + workerJvmMemoryOverheadMbs).append("M") .append(" -D").append(GobblinYarnConfigurationKeys.JVM_USER_TIMEZONE_CONFIG).append("=").append(this.containerTimezone) .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_DIR_NAME).append("=").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR) .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_FILE_NAME).append("=").append(containerProcessName).append(".").append(ApplicationConstants.STDOUT) - .append(" ").append(JvmUtils.formatJvmArguments(this.containerJvmArgs)) - .append(" ").append(this.proxyJvmArgs) + .append(" ").append(JvmUtils.formatJvmArguments(workerJvmArgs)) .append(" ").append(GobblinTemporalYarnTaskRunner.class.getName()) .append(" --").append(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME) .append(" ").append(this.applicationName) From 7e7113cafaa0e2f1d981f47eeb6677ca0554c936 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Mon, 18 Nov 2024 14:54:22 +0530 Subject: [PATCH 05/23] removed getter --- .../java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b5c1c4ab999..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 @@ -78,7 +78,7 @@ protected UnrecognizedProfile(ScalingDirective directive) { } private final WorkforceProfiles profiles; - @Getter private final WorkforceStaffing staffing; + private final WorkforceStaffing staffing; @Getter private volatile long lastRevisionEpochMillis; /** create new plan with the initial, baseline worker profile using `baselineConfig` at `initialSetPoint` */ From 0528f71a15db0d3bf1d2e3fc6f05141bbdce5e9e Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Tue, 19 Nov 2024 09:41:49 +0530 Subject: [PATCH 06/23] addressed few comments --- .../yarn/DynamicScalingYarnService.java | 24 +++++++++++++----- .../DynamicScalingYarnServiceManager.java | 25 ++++++++++++++++--- .../gobblin/temporal/yarn/YarnService.java | 23 ++++++++++++++--- 3 files changed, 59 insertions(+), 13 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 89fbc23ddd5..3322db49528 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -1,7 +1,23 @@ +/* + * 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.yarn; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.yarn.api.records.Resource; @@ -25,7 +41,6 @@ public class DynamicScalingYarnService extends YarnService { private final WorkforceStaffing workforceStaffing; private final WorkforcePlan workforcePlan; - private final AtomicLong allocationRequestIdGenerator = new AtomicLong(0L); public DynamicScalingYarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { @@ -33,8 +48,6 @@ public DynamicScalingYarnService(Config config, String applicationName, String a this.workforceStaffing = WorkforceStaffing.initialize(getInitialContainers()); this.workforcePlan = new WorkforcePlan(getConfig(), getInitialContainers()); - // Putting baseline profile in the map for default allocation request id (0) - this.allocationRequestIdToWorkerProfile.put(allocationRequestIdGenerator.getAndIncrement(), this.baselineWorkerProfile); } public synchronized void reviseWorkforcePlanAndRequestNewContainers(List scalingDirectives) { @@ -60,8 +73,7 @@ private synchronized void requestNewContainersForStaffingDeltas(StaffingDeltas d private synchronized void requestContainersForWorkerProfile(WorkerProfile workerProfile, int numContainers) { int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); - long allocationRequestId = allocationRequestIdGenerator.getAndIncrement(); - this.allocationRequestIdToWorkerProfile.put(allocationRequestId, workerProfile); + long allocationRequestId = generateAllocationRequestId(workerProfile); requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores), Optional.of(allocationRequestId)); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java index 4e98dcf8c52..f8ec09c6fa9 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java @@ -1,3 +1,20 @@ +/* + * 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.yarn; import java.io.IOException; @@ -60,23 +77,23 @@ protected void startUp() { Optional.ofNullable(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) ); - log.info("Starting the " + DynamicScalingYarnServiceManager.class.getSimpleName()); + log.info("Starting the " + this.getClass().getSimpleName()); log.info("Scheduling the dynamic scaling task with an interval of {} seconds", scheduleInterval); this.dynamicScalingExecutor.scheduleAtFixedRate( - new DynamicScalingYarnServiceRunnable(this.dynamicScalingYarnService, scalingDirectiveSource), + new GetScalingDirectivesRunnable(this.dynamicScalingYarnService, scalingDirectiveSource), initialDelay, scheduleInterval, TimeUnit.SECONDS ); } @Override protected void shutDown() { - log.info("Stopping the " + DynamicScalingYarnServiceManager.class.getSimpleName()); + log.info("Stopping the " + this.getClass().getSimpleName()); ExecutorsUtils.shutdownExecutorService(this.dynamicScalingExecutor, com.google.common.base.Optional.of(log)); } @AllArgsConstructor - static class DynamicScalingYarnServiceRunnable implements Runnable { + static class GetScalingDirectivesRunnable implements Runnable { private final DynamicScalingYarnService dynamicScalingYarnService; private final ScalingDirectiveSource scalingDirectiveSource; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index ae83402d12b..d7587799e51 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -34,6 +34,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; @@ -202,8 +203,9 @@ class YarnService extends AbstractIdleService { private volatile boolean shutdownInProgress = false; private final boolean jarCacheEnabled; - protected final WorkerProfile baselineWorkerProfile; - protected final ConcurrentMap allocationRequestIdToWorkerProfile = new ConcurrentHashMap<>(); + private final WorkerProfile baselineWorkerProfile; + private final AtomicLong allocationRequestIdGenerator = new AtomicLong(0L); + private final ConcurrentMap allocationRequestIdToWorkerProfile = new ConcurrentHashMap<>(); public YarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { @@ -282,6 +284,9 @@ public YarnService(Config config, String applicationName, String applicationId, // Initialising this baseline worker profile to use as default worker profile in case allocation request id is not in map this.baselineWorkerProfile = new WorkerProfile(WorkforceProfiles.BASELINE_NAME, this.config); + + // Putting baseline profile in the map for default allocation request id (0) + this.allocationRequestIdToWorkerProfile.put(allocationRequestIdGenerator.getAndIncrement(), this.baselineWorkerProfile); } @SuppressWarnings("unused") @@ -490,7 +495,7 @@ private void requestContainer(Optional preferredNode, Resource resource, String[] preferredNodes = preferredNode.isPresent() ? new String[] {preferredNode.get()} : null; - long allocationRequestID = allocationRequestId.isPresent() ? allocationRequestId.get() : 0L; + long allocationRequestID = allocationRequestId.or(0L); this.amrmClientAsync.addContainerRequest( new AMRMClient.ContainerRequest(resource, preferredNodes, null, priority, allocationRequestID)); @@ -778,6 +783,18 @@ private ImmutableMap.Builder buildContainerStatusEventMetadata(C return eventMetadataBuilder; } + /** + * Generates a unique allocation request ID for the given worker profile and store the id to profile mapping. + * + * @param workerProfile the worker profile for which the allocation request ID is generated + * @return the generated allocation request ID + */ + protected long generateAllocationRequestId(WorkerProfile workerProfile) { + long allocationRequestId = allocationRequestIdGenerator.getAndIncrement(); + this.allocationRequestIdToWorkerProfile.put(allocationRequestId, workerProfile); + return allocationRequestId; + } + /** * A custom implementation of {@link AMRMClientAsync.CallbackHandler}. */ From 8a51f6ca503987d4070850adf1d8b81341e6b745 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Sat, 23 Nov 2024 23:09:02 +0530 Subject: [PATCH 07/23] adding dummy scaling directive source for testing --- .../dynamic/DummyScalingDirectiveSource.java | 76 +++++++++++++++++++ .../yarn/DynamicScalingYarnService.java | 26 +++++-- .../DynamicScalingYarnServiceManager.java | 19 ++++- .../gobblin/temporal/yarn/YarnService.java | 29 +++---- 4 files changed, 124 insertions(+), 26 deletions(-) create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java new file mode 100644 index 00000000000..c3e1e8bd7e5 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.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.loadgen.dynamic; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; +import org.apache.gobblin.temporal.dynamic.ProfileDerivation; +import org.apache.gobblin.temporal.dynamic.ProfileOverlay; +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; +import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; + + +/** + * A dummy implementation of {@link ScalingDirectiveSource} that returns a fixed set of {@link ScalingDirective}s. + */ +public class DummyScalingDirectiveSource implements ScalingDirectiveSource { + private int count = 0; + private final Optional derivedFromBaseline; + public DummyScalingDirectiveSource() { + this.derivedFromBaseline = Optional.of(new ProfileDerivation(WorkforceProfiles.BASELINE_NAME, + new ProfileOverlay.Adding( + new ProfileOverlay.KVPair(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, "2048"), + new ProfileOverlay.KVPair(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, "2") + ) + )); + } + + /** + * @return {@link ScalingDirective}s - an impl. may choose to return all known directives or to give only newer + * directives than previously returned + */ + @Override + public List getScalingDirectives() { + // Note - profile should exist already pr is derived from other profile + if (this.count == 0) { + this.count++; + return Arrays.asList( + new ScalingDirective("firstProfile", 3, System.currentTimeMillis(), this.derivedFromBaseline), + new ScalingDirective("secondProfile", 2, System.currentTimeMillis(), this.derivedFromBaseline) + ); + } else if (this.count == 1) { + this.count++; + return Arrays.asList( + new ScalingDirective("firstProfile", 5, System.currentTimeMillis()), + new ScalingDirective("secondProfile", 3, System.currentTimeMillis()) + ); + } else if (this.count == 2) { + this.count++; + return Arrays.asList( + new ScalingDirective("firstProfile", 5, System.currentTimeMillis()), + new ScalingDirective("secondProfile", 3, System.currentTimeMillis()) + ); + } + return new ArrayList<>(); + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 3322db49528..4aad889b0ce 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -36,10 +36,16 @@ import org.apache.gobblin.temporal.dynamic.WorkforceStaffing; import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; +/** + * Service for dynamically scaling Gobblin containers running on YARN. + * This service manages workforce staffing and plans, and requests new containers as needed. + */ @Slf4j public class DynamicScalingYarnService extends YarnService { + /** this holds the current count of containers requested for each worker profile */ private final WorkforceStaffing workforceStaffing; + /** this holds the current total workforce plan as per latest received scaling directives */ private final WorkforcePlan workforcePlan; public DynamicScalingYarnService(Config config, String applicationName, String applicationId, @@ -50,21 +56,29 @@ public DynamicScalingYarnService(Config config, String applicationName, String a this.workforcePlan = new WorkforcePlan(getConfig(), getInitialContainers()); } + /** + * Revises the workforce plan and requests new containers based on the given scaling directives. + * + * @param scalingDirectives the list of scaling directives + */ public synchronized void reviseWorkforcePlanAndRequestNewContainers(List scalingDirectives) { this.workforcePlan.reviseWhenNewer(scalingDirectives); StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.workforceStaffing); requestNewContainersForStaffingDeltas(deltas); - // update our staffing after requesting new containers - scalingDirectives.forEach(directive -> this.workforceStaffing.reviseStaffing( - directive.getProfileName(), directive.getSetPoint(), directive.getTimestampEpochMillis()) - ); } private synchronized void requestNewContainersForStaffingDeltas(StaffingDeltas deltas) { deltas.getPerProfileDeltas().forEach(profileDelta -> { if (profileDelta.getDelta() > 0) { - log.info("Requesting {} new containers for profile {}", profileDelta.getDelta(), profileDelta.getProfile()); - requestContainersForWorkerProfile(profileDelta.getProfile(), profileDelta.getDelta()); + WorkerProfile workerProfile = profileDelta.getProfile(); + String profileName = workerProfile.getName(); + int curNumContainers = this.workforceStaffing.getStaffing(profileName).orElse(0); + int delta = profileDelta.getDelta(); + log.info("Requesting {} new containers for profile {} having currently {} containers", delta, + profileName, curNumContainers); + requestContainersForWorkerProfile(workerProfile, delta); + // update our staffing after requesting new containers + this.workforceStaffing.reviseStaffing(profileName, curNumContainers + delta, System.currentTimeMillis()); } // TODO: Decide how to handle negative deltas }); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java index f8ec09c6fa9..279a333069a 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java @@ -38,8 +38,13 @@ import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; import org.apache.gobblin.util.ConfigUtils; import org.apache.gobblin.util.ExecutorsUtils; +import org.apache.gobblin.temporal.loadgen.dynamic.DummyScalingDirectiveSource; +/** + * This class manages the dynamic scaling of the {@link YarnService} by periodically polling for scaling directives and passing + * the latest scaling directives to the {@link DynamicScalingYarnService} for processing. + */ @Slf4j public class DynamicScalingYarnServiceManager extends AbstractIdleService { @@ -71,12 +76,16 @@ protected void startUp() { int initialDelay = ConfigUtils.getInt(this.config, DYNAMIC_SCALING_INITIAL_DELAY, DEFAULT_DYNAMIC_SCALING_INITIAL_DELAY_SECS); - ScalingDirectiveSource scalingDirectiveSource = new FsScalingDirectiveSource( + ScalingDirectiveSource fsScalingDirectiveSource = new FsScalingDirectiveSource( this.fs, this.config.getString(DYNAMIC_SCALING_DIRECTIVES_DIR), Optional.ofNullable(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) ); + // TODO: remove this line later + // Using for testing purposes only + ScalingDirectiveSource scalingDirectiveSource = new DummyScalingDirectiveSource(); + log.info("Starting the " + this.getClass().getSimpleName()); log.info("Scheduling the dynamic scaling task with an interval of {} seconds", scheduleInterval); @@ -92,6 +101,10 @@ protected void shutDown() { ExecutorsUtils.shutdownExecutorService(this.dynamicScalingExecutor, com.google.common.base.Optional.of(log)); } + /** + * A {@link Runnable} that gets the scaling directives from the {@link ScalingDirectiveSource} and passes them to the + * {@link DynamicScalingYarnService} for processing. + */ @AllArgsConstructor static class GetScalingDirectivesRunnable implements Runnable { private final DynamicScalingYarnService dynamicScalingYarnService; @@ -101,7 +114,9 @@ static class GetScalingDirectivesRunnable implements Runnable { public void run() { try { List scalingDirectives = scalingDirectiveSource.getScalingDirectives(); - dynamicScalingYarnService.reviseWorkforcePlanAndRequestNewContainers(scalingDirectives); + if (!scalingDirectives.isEmpty()) { + dynamicScalingYarnService.reviseWorkforcePlanAndRequestNewContainers(scalingDirectives); + } } catch (IOException e) { log.error("Failed to get scaling directives", e); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index d7587799e51..ca7e2459a0e 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -154,8 +154,6 @@ class YarnService extends AbstractIdleService { private final int initialContainers; private final int requestedContainerMemoryMbs; private final int requestedContainerCores; - private final int jvmMemoryOverheadMbs; - private final double jvmMemoryXmxRatio; private final boolean containerHostAffinityEnabled; private final int helixInstanceMaxRetries; @@ -260,22 +258,6 @@ public YarnService(Config config, String applicationName, String applicationId, GobblinYarnConfigurationKeys.RELEASED_CONTAINERS_CACHE_EXPIRY_SECS, GobblinYarnConfigurationKeys.DEFAULT_RELEASED_CONTAINERS_CACHE_EXPIRY_SECS), TimeUnit.SECONDS).build(); - this.jvmMemoryXmxRatio = ConfigUtils.getDouble(this.config, - GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, - GobblinYarnConfigurationKeys.DEFAULT_CONTAINER_JVM_MEMORY_XMX_RATIO); - - Preconditions.checkArgument(this.jvmMemoryXmxRatio >= 0 && this.jvmMemoryXmxRatio <= 1, - GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY + " must be between 0 and 1 inclusive"); - - this.jvmMemoryOverheadMbs = ConfigUtils.getInt(this.config, - GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, - GobblinYarnConfigurationKeys.DEFAULT_CONTAINER_JVM_MEMORY_OVERHEAD_MBS); - - Preconditions.checkArgument(this.jvmMemoryOverheadMbs < this.requestedContainerMemoryMbs * this.jvmMemoryXmxRatio, - GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY + " cannot be more than " - + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY + " * " - + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY); - this.appViewAcl = ConfigUtils.getString(this.config, GobblinYarnConfigurationKeys.APP_VIEW_ACL, GobblinYarnConfigurationKeys.DEFAULT_APP_VIEW_ACL); this.containerTimezone = ConfigUtils.getString(this.config, GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_TIMEZONE, @@ -615,6 +597,17 @@ protected String buildContainerCommand(Container container, String helixParticip GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, GobblinYarnConfigurationKeys.DEFAULT_CONTAINER_JVM_MEMORY_OVERHEAD_MBS); + Preconditions.checkArgument(workerJvmMemoryXmxRatio >= 0 && workerJvmMemoryXmxRatio <= 1, + workerProfile.getName() + " : " + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY + + " must be between 0 and 1 inclusive"); + + long containerMemoryMbs = container.getResource().getMemorySize(); + + Preconditions.checkArgument(workerJvmMemoryOverheadMbs < containerMemoryMbs * workerJvmMemoryXmxRatio, + workerProfile.getName() + " : " + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY + + " cannot be more than " + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY + " * " + + GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY); + Optional workerJvmArgs = workerProfileConfig.hasPath(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY) ? Optional.of(workerProfileConfig.getString(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY)) : Optional.absent(); From 64eb6e260857824ad0b92795e7934499acdb7b44 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Sat, 23 Nov 2024 23:57:37 +0530 Subject: [PATCH 08/23] adding changes that was removed during rebase --- .../org/apache/gobblin/temporal/yarn/YarnService.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index ca7e2459a0e..c1a3826edff 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -37,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.IntStream; +import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -157,9 +158,8 @@ class YarnService extends AbstractIdleService { private final boolean containerHostAffinityEnabled; private final int helixInstanceMaxRetries; - - private final Optional containerJvmArgs; private final String containerTimezone; + private final String proxyJvmArgs; @Getter(AccessLevel.PROTECTED) private volatile Optional maxResourceCapacity = Optional.absent(); @@ -242,9 +242,8 @@ public YarnService(Config config, String applicationName, String applicationId, this.helixInstanceTags = ConfigUtils.getString(config, GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, GobblinClusterConfigurationKeys.HELIX_DEFAULT_TAG); - this.containerJvmArgs = config.hasPath(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY) ? - Optional.of(config.getString(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY)) : - Optional.absent(); + this.proxyJvmArgs = config.hasPath(GobblinYarnConfigurationKeys.YARN_APPLICATION_PROXY_JVM_ARGS) ? + config.getString(GobblinYarnConfigurationKeys.YARN_APPLICATION_PROXY_JVM_ARGS) : StringUtils.EMPTY; int numContainerLaunchThreads = ConfigUtils.getInt(config, GobblinYarnConfigurationKeys.MAX_CONTAINER_LAUNCH_THREADS_KEY, @@ -621,6 +620,7 @@ protected String buildContainerCommand(Container container, String helixParticip .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_DIR_NAME).append("=").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR) .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_FILE_NAME).append("=").append(containerProcessName).append(".").append(ApplicationConstants.STDOUT) .append(" ").append(JvmUtils.formatJvmArguments(workerJvmArgs)) + .append(" ").append(this.proxyJvmArgs) .append(" ").append(GobblinTemporalYarnTaskRunner.class.getName()) .append(" --").append(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME) .append(" ").append(this.applicationName) From 709402388e9cf6bfe8a20876d405fd115b33fbbc Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Thu, 28 Nov 2024 21:32:29 +0530 Subject: [PATCH 09/23] addressed few review comments --- .../yarn/DynamicScalingYarnService.java | 13 +++++---- .../DynamicScalingYarnServiceManager.java | 29 +++++++++++-------- .../gobblin/temporal/yarn/YarnService.java | 26 +++++++++-------- 3 files changed, 39 insertions(+), 29 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 4aad889b0ce..e46e4777fa0 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -72,22 +72,25 @@ private synchronized void requestNewContainersForStaffingDeltas(StaffingDeltas d if (profileDelta.getDelta() > 0) { WorkerProfile workerProfile = profileDelta.getProfile(); String profileName = workerProfile.getName(); - int curNumContainers = this.workforceStaffing.getStaffing(profileName).orElse(0); + int currNumContainers = this.workforceStaffing.getStaffing(profileName).orElse(0); int delta = profileDelta.getDelta(); log.info("Requesting {} new containers for profile {} having currently {} containers", delta, - profileName, curNumContainers); + profileName, currNumContainers); requestContainersForWorkerProfile(workerProfile, delta); // update our staffing after requesting new containers - this.workforceStaffing.reviseStaffing(profileName, curNumContainers + delta, System.currentTimeMillis()); + this.workforceStaffing.reviseStaffing(profileName, currNumContainers + delta, System.currentTimeMillis()); + } else { + // TODO: Decide how to handle negative deltas + log.warn("Handling of Negative delta is not supported yet : Profile {} delta {} ", + profileDelta.getProfile().getName(), profileDelta.getDelta()); } - // TODO: Decide how to handle negative deltas }); } private synchronized void requestContainersForWorkerProfile(WorkerProfile workerProfile, int numContainers) { int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); - long allocationRequestId = generateAllocationRequestId(workerProfile); + long allocationRequestId = storeByUniqueAllocationRequestId(workerProfile); requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores), Optional.of(allocationRequestId)); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java index 279a333069a..5914193a8c3 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java @@ -51,18 +51,23 @@ public class DynamicScalingYarnServiceManager extends AbstractIdleService { private final String DYNAMIC_SCALING_PREFIX = GobblinTemporalConfigurationKeys.PREFIX + "dynamic.scaling."; private final String DYNAMIC_SCALING_DIRECTIVES_DIR = DYNAMIC_SCALING_PREFIX + "directives.dir"; private final String DYNAMIC_SCALING_ERRORS_DIR = DYNAMIC_SCALING_PREFIX + "errors.dir"; - private final String DYNAMIC_SCALING_INITIAL_DELAY = DYNAMIC_SCALING_PREFIX + "initial.delay"; - private final int DEFAULT_DYNAMIC_SCALING_INITIAL_DELAY_SECS = 60; private final String DYNAMIC_SCALING_POLLING_INTERVAL = DYNAMIC_SCALING_PREFIX + "polling.interval"; private final int DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS = 60; private final Config config; - DynamicScalingYarnService dynamicScalingYarnService; + private final DynamicScalingYarnService dynamicScalingYarnService; private final ScheduledExecutorService dynamicScalingExecutor; private final FileSystem fs; public DynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { this.config = appMaster.getConfig(); - this.dynamicScalingYarnService = (DynamicScalingYarnService) appMaster.get_yarnService(); + if (appMaster.get_yarnService() instanceof DynamicScalingYarnService) { + this.dynamicScalingYarnService = (DynamicScalingYarnService) appMaster.get_yarnService(); + } else { + String errorMsg = "Failure while getting YarnService Instance from GobblinTemporalApplicationMaster::get_yarnService()" + + " YarnService is not an instance of DynamicScalingYarnService"; + log.error(errorMsg); + throw new RuntimeException(errorMsg); + } this.dynamicScalingExecutor = Executors.newSingleThreadScheduledExecutor( ExecutorsUtils.newThreadFactory(com.google.common.base.Optional.of(log), com.google.common.base.Optional.of("DynamicScalingExecutor"))); @@ -73,14 +78,12 @@ public DynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMast protected void startUp() { int scheduleInterval = ConfigUtils.getInt(this.config, DYNAMIC_SCALING_POLLING_INTERVAL, DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS); - int initialDelay = ConfigUtils.getInt(this.config, DYNAMIC_SCALING_INITIAL_DELAY, - DEFAULT_DYNAMIC_SCALING_INITIAL_DELAY_SECS); - ScalingDirectiveSource fsScalingDirectiveSource = new FsScalingDirectiveSource( - this.fs, - this.config.getString(DYNAMIC_SCALING_DIRECTIVES_DIR), - Optional.ofNullable(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) - ); +// ScalingDirectiveSource fsScalingDirectiveSource = new FsScalingDirectiveSource( +// this.fs, +// this.config.getString(DYNAMIC_SCALING_DIRECTIVES_DIR), +// Optional.ofNullable(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) +// ); // TODO: remove this line later // Using for testing purposes only @@ -91,7 +94,7 @@ protected void startUp() { this.dynamicScalingExecutor.scheduleAtFixedRate( new GetScalingDirectivesRunnable(this.dynamicScalingYarnService, scalingDirectiveSource), - initialDelay, scheduleInterval, TimeUnit.SECONDS + scheduleInterval, scheduleInterval, TimeUnit.SECONDS ); } @@ -119,6 +122,8 @@ public void run() { } } catch (IOException e) { log.error("Failed to get scaling directives", e); + } catch (Throwable t) { + log.error("Suppressing error from GetScalingDirectivesRunnable.run()", t); } } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index c1a3826edff..ca3cf0e8d75 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -203,7 +203,7 @@ class YarnService extends AbstractIdleService { private final boolean jarCacheEnabled; private final WorkerProfile baselineWorkerProfile; private final AtomicLong allocationRequestIdGenerator = new AtomicLong(0L); - private final ConcurrentMap allocationRequestIdToWorkerProfile = new ConcurrentHashMap<>(); + private final ConcurrentMap workerProfileByAllocationRequestId = new ConcurrentHashMap<>(); public YarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { @@ -267,7 +267,7 @@ public YarnService(Config config, String applicationName, String applicationId, this.baselineWorkerProfile = new WorkerProfile(WorkforceProfiles.BASELINE_NAME, this.config); // Putting baseline profile in the map for default allocation request id (0) - this.allocationRequestIdToWorkerProfile.put(allocationRequestIdGenerator.getAndIncrement(), this.baselineWorkerProfile); + this.workerProfileByAllocationRequestId.put(allocationRequestIdGenerator.getAndIncrement(), this.baselineWorkerProfile); } @SuppressWarnings("unused") @@ -452,14 +452,14 @@ private void requestContainer(Optional preferredNode, Optional * @param numContainers * @param resource */ - protected void requestContainers(int numContainers, Resource resource, Optional allocationRequestId) { - LOGGER.info("Requesting {} containers with resource={} and allocation request id = {}", numContainers, resource, allocationRequestId); + protected void requestContainers(int numContainers, Resource resource, Optional optAllocationRequestId) { + LOGGER.info("Requesting {} containers with resource={} and allocation request id = {}", numContainers, resource, optAllocationRequestId); IntStream.range(0, numContainers) - .forEach(i -> requestContainer(Optional.absent(), resource, allocationRequestId)); + .forEach(i -> requestContainer(Optional.absent(), resource, optAllocationRequestId)); } // Request containers with specific resource requirement - private void requestContainer(Optional preferredNode, Resource resource, Optional allocationRequestId) { + private void requestContainer(Optional preferredNode, Resource resource, Optional optAllocationRequestId) { // Fail if Yarn cannot meet container resource requirements Preconditions.checkArgument(resource.getMemory() <= this.maxResourceCapacity.get().getMemory() && resource.getVirtualCores() <= this.maxResourceCapacity.get().getVirtualCores(), @@ -476,10 +476,10 @@ private void requestContainer(Optional preferredNode, Resource resource, String[] preferredNodes = preferredNode.isPresent() ? new String[] {preferredNode.get()} : null; - long allocationRequestID = allocationRequestId.or(0L); + long allocationRequestId = optAllocationRequestId.or(0L); this.amrmClientAsync.addContainerRequest( - new AMRMClient.ContainerRequest(resource, preferredNodes, null, priority, allocationRequestID)); + new AMRMClient.ContainerRequest(resource, preferredNodes, null, priority, allocationRequestId)); } protected ContainerLaunchContext newContainerLaunchContext(ContainerInfo containerInfo) @@ -583,8 +583,9 @@ protected ByteBuffer getSecurityTokens() throws IOException { @VisibleForTesting protected String buildContainerCommand(Container container, String helixParticipantId, String helixInstanceTag) { - long allocationRequestID = container.getAllocationRequestId(); - WorkerProfile workerProfile = this.allocationRequestIdToWorkerProfile.getOrDefault(allocationRequestID, + long allocationRequestId = container.getAllocationRequestId(); + // Using getOrDefault for backward-compatibility with containers that don't have allocationRequestId set + WorkerProfile workerProfile = this.workerProfileByAllocationRequestId.getOrDefault(allocationRequestId, this.baselineWorkerProfile); Config workerProfileConfig = workerProfile.getConfig(); @@ -782,9 +783,9 @@ private ImmutableMap.Builder buildContainerStatusEventMetadata(C * @param workerProfile the worker profile for which the allocation request ID is generated * @return the generated allocation request ID */ - protected long generateAllocationRequestId(WorkerProfile workerProfile) { + protected long storeByUniqueAllocationRequestId(WorkerProfile workerProfile) { long allocationRequestId = allocationRequestIdGenerator.getAndIncrement(); - this.allocationRequestIdToWorkerProfile.put(allocationRequestId, workerProfile); + this.workerProfileByAllocationRequestId.put(allocationRequestId, workerProfile); return allocationRequestId; } @@ -838,6 +839,7 @@ public void onContainersAllocated(List containers) { // YARN does not have a delta request API and the requests are not cleaned up automatically. // Try finding a match first with the host as the resource name then fall back to any resource match. // Also see YARN-1902. Container count will explode without this logic for removing container requests. + // TODO : Add removing by allocation request id first List> matchingRequests = amrmClientAsync .getMatchingRequests(container.getPriority(), container.getNodeHttpAddress(), container.getResource()); From 9ccf8ae1fe209d7dc99f53ce6dcb91e5bee3fa04 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Thu, 28 Nov 2024 23:06:20 +0530 Subject: [PATCH 10/23] changed initial container request to use baseline worker profile --- .../yarn/DynamicScalingYarnService.java | 6 ++- .../gobblin/temporal/yarn/YarnService.java | 51 +++++-------------- 2 files changed, 16 insertions(+), 41 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index e46e4777fa0..a2351eb1d97 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -52,8 +52,10 @@ public DynamicScalingYarnService(Config config, String applicationName, String a YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { super(config, applicationName, applicationId, yarnConfiguration, fs, eventBus); - this.workforceStaffing = WorkforceStaffing.initialize(getInitialContainers()); - this.workforcePlan = new WorkforcePlan(getConfig(), getInitialContainers()); + int initialContainers = this.baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); + + this.workforceStaffing = WorkforceStaffing.initialize(initialContainers); + this.workforcePlan = new WorkforcePlan(this.baselineWorkerProfile.getConfig(), initialContainers); } /** diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index ca3cf0e8d75..2c942dfb3b0 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -133,8 +133,6 @@ class YarnService extends AbstractIdleService { private final String appViewAcl; //Default helix instance tag derived from cluster level config private final String helixInstanceTags; - - @Getter(AccessLevel.PROTECTED) private final Config config; private final EventBus eventBus; @@ -150,9 +148,6 @@ class YarnService extends AbstractIdleService { private final AMRMClientAsync amrmClientAsync; private final NMClientAsync nmClientAsync; private final ExecutorService containerLaunchExecutor; - - @Getter(AccessLevel.PROTECTED) - private final int initialContainers; private final int requestedContainerMemoryMbs; private final int requestedContainerCores; private final boolean containerHostAffinityEnabled; @@ -201,7 +196,7 @@ class YarnService extends AbstractIdleService { private volatile boolean shutdownInProgress = false; private final boolean jarCacheEnabled; - private final WorkerProfile baselineWorkerProfile; + protected final WorkerProfile baselineWorkerProfile; private final AtomicLong allocationRequestIdGenerator = new AtomicLong(0L); private final ConcurrentMap workerProfileByAllocationRequestId = new ConcurrentHashMap<>(); @@ -233,7 +228,6 @@ public YarnService(Config config, String applicationName, String applicationId, this.nmClientAsync = closer.register(NMClientAsync.createNMClientAsync(getNMClientCallbackHandler())); this.nmClientAsync.init(this.yarnConfiguration); - this.initialContainers = config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); this.requestedContainerMemoryMbs = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); this.requestedContainerCores = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); this.containerHostAffinityEnabled = config.getBoolean(GobblinYarnConfigurationKeys.CONTAINER_HOST_AFFINITY_ENABLED); @@ -267,7 +261,8 @@ public YarnService(Config config, String applicationName, String applicationId, this.baselineWorkerProfile = new WorkerProfile(WorkforceProfiles.BASELINE_NAME, this.config); // Putting baseline profile in the map for default allocation request id (0) - this.workerProfileByAllocationRequestId.put(allocationRequestIdGenerator.getAndIncrement(), this.baselineWorkerProfile); + // adding it here to have deterministic allocation request id for baseline worker profile + storeByUniqueAllocationRequestId(this.baselineWorkerProfile); } @SuppressWarnings("unused") @@ -333,8 +328,8 @@ protected synchronized void startUp() throws Exception { LOGGER.info("ApplicationMaster registration response: " + response); this.maxResourceCapacity = Optional.of(response.getMaximumResourceCapability()); - LOGGER.info("Requesting initial containers"); - requestInitialContainers(this.initialContainers); + LOGGER.info("Requesting initial containers using baselineWorkerProfile"); + requestInitialContainers(this.baselineWorkerProfile); } @Override @@ -409,35 +404,13 @@ private EventSubmitter buildEventSubmitter() { .build(); } - /** - * Request an allocation of containers. If numTargetContainers is larger than the max of current and expected number - * of containers then additional containers are requested. - *

- * If numTargetContainers is less than the current number of allocated containers then release free containers. - * Shrinking is relative to the number of currently allocated containers since it takes time for containers - * to be allocated and assigned work and we want to avoid releasing a container prematurely before it is assigned - * work. This means that a container may not be released even though numTargetContainers is less than the requested - * number of containers. The intended usage is for the caller of this method to make periodic calls to attempt to - * adjust the cluster towards the desired number of containers. - * - * @param inUseInstances a set of in use instances - * @return whether successfully requested the target number of containers - */ - public synchronized boolean requestTargetNumberOfContainers(int numContainers, Set inUseInstances) { - int defaultContainerMemoryMbs = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); - int defaultContainerCores = config.getInt(GobblinYarnConfigurationKeys. CONTAINER_CORES_KEY); - - LOGGER.info("Trying to set numTargetContainers={}, in-use helix instances count is {}, container map size is {}", - numContainers, inUseInstances.size(), this.containerMap.size()); - - requestContainers(numContainers, Resource.newInstance(defaultContainerMemoryMbs, defaultContainerCores), Optional.absent()); - LOGGER.info("Current tag-container desired count:{}, tag-container allocated: {}", numContainers, this.allocatedContainerCountMap); - return true; - } - - // Request initial containers with default resource and helix tag - private void requestInitialContainers(int containersRequested) { - requestTargetNumberOfContainers(containersRequested, Collections.EMPTY_SET); + /** Request Initial containers using baselineWorkerProfile */ + private void requestInitialContainers(WorkerProfile workerProfile) { + int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); + int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); + int numContainers = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); + // Using 0 as allocation id for baseline worker profile + requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores), Optional.of(0L)); } private void requestContainer(Optional preferredNode, Optional resourceOptional) { From 606349eac787402beb0bebca1df14646f5d8cbae Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Fri, 29 Nov 2024 19:07:25 +0530 Subject: [PATCH 11/23] added abstractdynamicscalingmanger --- ...DummyDynamicScalingYarnServiceManager.java | 38 +++++++++++++++ .../dynamic/DummyScalingDirectiveSource.java | 36 +++++++++----- ...ractDynamicScalingYarnServiceManager.java} | 47 +++++++------------ ...ourceDynamicScalingYarnServiceManager.java | 47 +++++++++++++++++++ .../gobblin/temporal/yarn/YarnService.java | 37 +++++++++------ 5 files changed, 148 insertions(+), 57 deletions(-) create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyDynamicScalingYarnServiceManager.java rename gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/{DynamicScalingYarnServiceManager.java => AbstractDynamicScalingYarnServiceManager.java} (78%) create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyDynamicScalingYarnServiceManager.java new file mode 100644 index 00000000000..a4ec19b587f --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyDynamicScalingYarnServiceManager.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.temporal.loadgen.dynamic; + +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; +import org.apache.gobblin.temporal.yarn.GobblinTemporalApplicationMaster; +import org.apache.gobblin.temporal.yarn.AbstractDynamicScalingYarnServiceManager; + +/** + * {@link DummyScalingDirectiveSource} based implementation of {@link AbstractDynamicScalingYarnServiceManager}. + * This class is meant to be used for testing purposes only. + */ +public class DummyDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { + + public DummyDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { + super(appMaster); + } + + @Override + protected ScalingDirectiveSource createScalingDirectiveSource() { + return new DummyScalingDirectiveSource(); + } +} \ No newline at end of file diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java index c3e1e8bd7e5..847919abb03 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; import org.apache.gobblin.temporal.dynamic.ProfileDerivation; @@ -34,7 +35,7 @@ * A dummy implementation of {@link ScalingDirectiveSource} that returns a fixed set of {@link ScalingDirective}s. */ public class DummyScalingDirectiveSource implements ScalingDirectiveSource { - private int count = 0; + private final AtomicInteger numInvocations = new AtomicInteger(0); private final Optional derivedFromBaseline; public DummyScalingDirectiveSource() { this.derivedFromBaseline = Optional.of(new ProfileDerivation(WorkforceProfiles.BASELINE_NAME, @@ -52,23 +53,32 @@ public DummyScalingDirectiveSource() { @Override public List getScalingDirectives() { // Note - profile should exist already pr is derived from other profile - if (this.count == 0) { - this.count++; + if (this.numInvocations.get() == 0) { + this.numInvocations.getAndIncrement(); + // here we are returning two new profile with initial container counts and these should be launched + long currentTime = System.currentTimeMillis(); + // both profiles should have different timestampEpochMillis so that both are processed otherwise + // }org.apache.gobblin.temporal.dynamic.WorkforcePlan$IllegalRevisionException$OutOfOrderDirective can occur return Arrays.asList( - new ScalingDirective("firstProfile", 3, System.currentTimeMillis(), this.derivedFromBaseline), - new ScalingDirective("secondProfile", 2, System.currentTimeMillis(), this.derivedFromBaseline) + new ScalingDirective("firstProfile", 3, currentTime, this.derivedFromBaseline), + new ScalingDirective("secondProfile", 2, currentTime + 1, this.derivedFromBaseline) ); - } else if (this.count == 1) { - this.count++; + } else if (this.numInvocations.get() == 1) { + this.numInvocations.getAndIncrement(); + // here we are increasing containers to 5 for firstProfile and 3 for secondProfile so that 2 new extra containers + // should be launched for firstProfile and 1 new extra container for secondProfile + long currentTime = System.currentTimeMillis(); return Arrays.asList( - new ScalingDirective("firstProfile", 5, System.currentTimeMillis()), - new ScalingDirective("secondProfile", 3, System.currentTimeMillis()) + new ScalingDirective("firstProfile", 5, currentTime), + new ScalingDirective("secondProfile", 3, currentTime + 1) ); - } else if (this.count == 2) { - this.count++; + } else if (this.numInvocations.get() == 2) { + this.numInvocations.getAndIncrement(); + // the count is same as previous invocation so no new containers should be launched + long currentTime = System.currentTimeMillis(); return Arrays.asList( - new ScalingDirective("firstProfile", 5, System.currentTimeMillis()), - new ScalingDirective("secondProfile", 3, System.currentTimeMillis()) + new ScalingDirective("firstProfile", 5, currentTime), + new ScalingDirective("secondProfile", 3, currentTime + 1) ); } return new ArrayList<>(); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java similarity index 78% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java rename to gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java index 5914193a8c3..545e6224957 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java @@ -19,12 +19,9 @@ import java.io.IOException; import java.util.List; -import java.util.Optional; +import java.util.concurrent.TimeUnit; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.fs.FileSystem; import com.typesafe.config.Config; import com.google.common.util.concurrent.AbstractIdleService; @@ -32,33 +29,30 @@ import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; -import org.apache.gobblin.temporal.dynamic.FsScalingDirectiveSource; -import org.apache.gobblin.temporal.dynamic.ScalingDirective; -import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; import org.apache.gobblin.util.ConfigUtils; import org.apache.gobblin.util.ExecutorsUtils; -import org.apache.gobblin.temporal.loadgen.dynamic.DummyScalingDirectiveSource; - +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; /** * This class manages the dynamic scaling of the {@link YarnService} by periodically polling for scaling directives and passing * the latest scaling directives to the {@link DynamicScalingYarnService} for processing. + * + * This is an abstract class that provides the basic functionality for managing dynamic scaling. Subclasses should implement + * {@link #createScalingDirectiveSource()} to provide a {@link ScalingDirectiveSource} that will be used to get scaling directives. */ @Slf4j -public class DynamicScalingYarnServiceManager extends AbstractIdleService { +public abstract class AbstractDynamicScalingYarnServiceManager extends AbstractIdleService { - private final String DYNAMIC_SCALING_PREFIX = GobblinTemporalConfigurationKeys.PREFIX + "dynamic.scaling."; - private final String DYNAMIC_SCALING_DIRECTIVES_DIR = DYNAMIC_SCALING_PREFIX + "directives.dir"; - private final String DYNAMIC_SCALING_ERRORS_DIR = DYNAMIC_SCALING_PREFIX + "errors.dir"; + protected final String DYNAMIC_SCALING_PREFIX = GobblinTemporalConfigurationKeys.PREFIX + "dynamic.scaling."; private final String DYNAMIC_SCALING_POLLING_INTERVAL = DYNAMIC_SCALING_PREFIX + "polling.interval"; private final int DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS = 60; - private final Config config; + protected final Config config; private final DynamicScalingYarnService dynamicScalingYarnService; private final ScheduledExecutorService dynamicScalingExecutor; - private final FileSystem fs; - public DynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { + public AbstractDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { this.config = appMaster.getConfig(); if (appMaster.get_yarnService() instanceof DynamicScalingYarnService) { this.dynamicScalingYarnService = (DynamicScalingYarnService) appMaster.get_yarnService(); @@ -71,29 +65,17 @@ public DynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMast this.dynamicScalingExecutor = Executors.newSingleThreadScheduledExecutor( ExecutorsUtils.newThreadFactory(com.google.common.base.Optional.of(log), com.google.common.base.Optional.of("DynamicScalingExecutor"))); - this.fs = appMaster.getFs(); } @Override protected void startUp() { int scheduleInterval = ConfigUtils.getInt(this.config, DYNAMIC_SCALING_POLLING_INTERVAL, DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS); - -// ScalingDirectiveSource fsScalingDirectiveSource = new FsScalingDirectiveSource( -// this.fs, -// this.config.getString(DYNAMIC_SCALING_DIRECTIVES_DIR), -// Optional.ofNullable(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) -// ); - - // TODO: remove this line later - // Using for testing purposes only - ScalingDirectiveSource scalingDirectiveSource = new DummyScalingDirectiveSource(); - log.info("Starting the " + this.getClass().getSimpleName()); log.info("Scheduling the dynamic scaling task with an interval of {} seconds", scheduleInterval); this.dynamicScalingExecutor.scheduleAtFixedRate( - new GetScalingDirectivesRunnable(this.dynamicScalingYarnService, scalingDirectiveSource), + new GetScalingDirectivesRunnable(this.dynamicScalingYarnService, createScalingDirectiveSource()), scheduleInterval, scheduleInterval, TimeUnit.SECONDS ); } @@ -104,6 +86,11 @@ protected void shutDown() { ExecutorsUtils.shutdownExecutorService(this.dynamicScalingExecutor, com.google.common.base.Optional.of(log)); } + /** + * Create a {@link ScalingDirectiveSource} to use for getting scaling directives. + */ + protected abstract ScalingDirectiveSource createScalingDirectiveSource(); + /** * A {@link Runnable} that gets the scaling directives from the {@link ScalingDirectiveSource} and passes them to the * {@link DynamicScalingYarnService} for processing. diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java new file mode 100644 index 00000000000..9aa5bd1eb57 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java @@ -0,0 +1,47 @@ +/* + * 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.yarn; + +import java.util.Optional; +import org.apache.gobblin.temporal.dynamic.FsScalingDirectiveSource; +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; +import org.apache.hadoop.fs.FileSystem; + + +/** + * {@link FsScalingDirectiveSource} based implementation of {@link AbstractDynamicScalingYarnServiceManager}. + */ +public class FsSourceDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { + private final String DYNAMIC_SCALING_DIRECTIVES_DIR = DYNAMIC_SCALING_PREFIX + "directives.dir"; + private final String DYNAMIC_SCALING_ERRORS_DIR = DYNAMIC_SCALING_PREFIX + "errors.dir"; + private final FileSystem fs; + + public FsSourceDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { + super(appMaster); + this.fs = appMaster.getFs(); + } + + @Override + protected ScalingDirectiveSource createScalingDirectiveSource() { + return new FsScalingDirectiveSource( + this.fs, + this.config.getString(DYNAMIC_SCALING_DIRECTIVES_DIR), + Optional.ofNullable(this.config.getString(DYNAMIC_SCALING_ERRORS_DIR)) + ); + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index 2c942dfb3b0..efe9423fe80 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -23,7 +23,6 @@ import java.time.Duration; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -812,23 +811,33 @@ public void onContainersAllocated(List containers) { // YARN does not have a delta request API and the requests are not cleaned up automatically. // Try finding a match first with the host as the resource name then fall back to any resource match. // Also see YARN-1902. Container count will explode without this logic for removing container requests. - // TODO : Add removing by allocation request id first - List> matchingRequests = amrmClientAsync - .getMatchingRequests(container.getPriority(), container.getNodeHttpAddress(), container.getResource()); + Collection matchingRequestsByAllocationRequestId = amrmClientAsync.getMatchingRequests(container.getAllocationRequestId()); + if (!matchingRequestsByAllocationRequestId.isEmpty()) { + AMRMClient.ContainerRequest firstMatchingContainerRequest = matchingRequestsByAllocationRequestId.iterator().next(); + LOGGER.debug("Found matching requests {}, removing first matching request {}", + matchingRequestsByAllocationRequestId, firstMatchingContainerRequest); - if (matchingRequests.isEmpty()) { - LOGGER.debug("Matching request by host {} not found", container.getNodeHttpAddress()); + amrmClientAsync.removeContainerRequest(firstMatchingContainerRequest); + } else { + LOGGER.debug("Matching request by allocation request id {} not found", container.getAllocationRequestId()); - matchingRequests = amrmClientAsync - .getMatchingRequests(container.getPriority(), ResourceRequest.ANY, container.getResource()); - } + List> matchingRequestsByHost = amrmClientAsync + .getMatchingRequests(container.getPriority(), container.getNodeHttpAddress(), container.getResource()); - if (!matchingRequests.isEmpty()) { - AMRMClient.ContainerRequest firstMatchingContainerRequest = matchingRequests.get(0).iterator().next(); - LOGGER.debug("Found matching requests {}, removing first matching request {}", - matchingRequests, firstMatchingContainerRequest); + if (matchingRequestsByHost.isEmpty()) { + LOGGER.debug("Matching request by host {} not found", container.getNodeHttpAddress()); - amrmClientAsync.removeContainerRequest(firstMatchingContainerRequest); + matchingRequestsByHost = amrmClientAsync + .getMatchingRequests(container.getPriority(), ResourceRequest.ANY, container.getResource()); + } + + if (!matchingRequestsByHost.isEmpty()) { + AMRMClient.ContainerRequest firstMatchingContainerRequest = matchingRequestsByHost.get(0).iterator().next(); + LOGGER.debug("Found matching requests {}, removing first matching request {}", + matchingRequestsByAllocationRequestId, firstMatchingContainerRequest); + + amrmClientAsync.removeContainerRequest(firstMatchingContainerRequest); + } } containerLaunchExecutor.submit(new Runnable() { From 0c87a75b596b37f564976fccf5e6c620f4c212d7 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Fri, 29 Nov 2024 19:47:10 +0530 Subject: [PATCH 12/23] fix typo and added one extra log line while launching initial containers --- .../loadgen/dynamic/DummyScalingDirectiveSource.java | 7 +++---- .../java/org/apache/gobblin/temporal/yarn/YarnService.java | 1 + 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java index 847919abb03..fc5bb50f5a6 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java @@ -47,18 +47,17 @@ public DummyScalingDirectiveSource() { } /** - * @return {@link ScalingDirective}s - an impl. may choose to return all known directives or to give only newer - * directives than previously returned + * @return - A fixed set of {@link ScalingDirective}s corresponding to the invocation number. */ @Override public List getScalingDirectives() { - // Note - profile should exist already pr is derived from other profile + // Note - profile should exist already or is derived from other profile if (this.numInvocations.get() == 0) { this.numInvocations.getAndIncrement(); // here we are returning two new profile with initial container counts and these should be launched long currentTime = System.currentTimeMillis(); // both profiles should have different timestampEpochMillis so that both are processed otherwise - // }org.apache.gobblin.temporal.dynamic.WorkforcePlan$IllegalRevisionException$OutOfOrderDirective can occur + // org.apache.gobblin.temporal.dynamic.WorkforcePlan$IllegalRevisionException$OutOfOrderDirective can occur return Arrays.asList( new ScalingDirective("firstProfile", 3, currentTime, this.derivedFromBaseline), new ScalingDirective("secondProfile", 2, currentTime + 1, this.derivedFromBaseline) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index efe9423fe80..6fa64815f9b 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -408,6 +408,7 @@ private void requestInitialContainers(WorkerProfile workerProfile) { int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); int numContainers = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); + LOGGER.info("Requesting {} initial containers with default allocation id = 0", numContainers); // Using 0 as allocation id for baseline worker profile requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores), Optional.of(0L)); } From c7a6d1ab737119c8a5c38adf7d5b0869e8b4e8fa Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Fri, 29 Nov 2024 23:28:37 +0530 Subject: [PATCH 13/23] added unit test for buildcontainercommand --- .../GobblinTemporalConfigurationKeys.java | 5 ++ ...tractDynamicScalingYarnServiceManager.java | 3 +- ...ourceDynamicScalingYarnServiceManager.java | 10 ++- .../temporal/yarn/YarnServiceTest.java | 77 +++++++++++++++++++ .../src/test/resources/YarnServiceTest.conf | 5 ++ 5 files changed, 94 insertions(+), 6 deletions(-) create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java create mode 100644 gobblin-temporal/src/test/resources/YarnServiceTest.conf diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java index 40223e093e8..3d51f15c19d 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java @@ -64,4 +64,9 @@ public interface GobblinTemporalConfigurationKeys { String TEMPORAL_NUM_WORKERS_PER_CONTAINER = PREFIX + "num.workers.per.container"; int DEFAULT_TEMPORAL_NUM_WORKERS_PER_CONTAINERS = 1; String TEMPORAL_CONNECTION_STRING = PREFIX + "connection.string"; + + /** + * Prefix for Gobblin-on-Temporal Dynamic Scaling + */ + String DYNAMIC_SCALING_PREFIX = PREFIX + "dynamic.scaling."; } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java index 545e6224957..bb0853196c6 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java @@ -45,8 +45,7 @@ @Slf4j public abstract class AbstractDynamicScalingYarnServiceManager extends AbstractIdleService { - protected final String DYNAMIC_SCALING_PREFIX = GobblinTemporalConfigurationKeys.PREFIX + "dynamic.scaling."; - private final String DYNAMIC_SCALING_POLLING_INTERVAL = DYNAMIC_SCALING_PREFIX + "polling.interval"; + private final static String DYNAMIC_SCALING_POLLING_INTERVAL = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "polling.interval"; private final int DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS = 60; protected final Config config; private final DynamicScalingYarnService dynamicScalingYarnService; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java index 9aa5bd1eb57..88130ce9cc2 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java @@ -18,17 +18,19 @@ package org.apache.gobblin.temporal.yarn; import java.util.Optional; -import org.apache.gobblin.temporal.dynamic.FsScalingDirectiveSource; -import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; + import org.apache.hadoop.fs.FileSystem; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.dynamic.FsScalingDirectiveSource; +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; /** * {@link FsScalingDirectiveSource} based implementation of {@link AbstractDynamicScalingYarnServiceManager}. */ public class FsSourceDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { - private final String DYNAMIC_SCALING_DIRECTIVES_DIR = DYNAMIC_SCALING_PREFIX + "directives.dir"; - private final String DYNAMIC_SCALING_ERRORS_DIR = DYNAMIC_SCALING_PREFIX + "errors.dir"; + private final static String DYNAMIC_SCALING_DIRECTIVES_DIR = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "directives.dir"; + private final static String DYNAMIC_SCALING_ERRORS_DIR = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "errors.dir"; private final FileSystem fs; public FsSourceDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java new file mode 100644 index 00000000000..e5c4f957815 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java @@ -0,0 +1,77 @@ +/* + * 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.yarn; + +import com.google.common.eventbus.EventBus; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; +import org.apache.hadoop.yarn.api.records.Container; +import java.net.URL; +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +/** Tests for {@link YarnService}*/ +public class YarnServiceTest { + private Config defaultConfigs; + private YarnConfiguration yarnConfiguration = new YarnConfiguration(); + private final EventBus eventBus = new EventBus("TemporalYarnServiceTest"); + + @BeforeClass + public void setup() { + URL url = YarnServiceTest.class.getClassLoader() + .getResource(YarnServiceTest.class.getSimpleName() + ".conf"); + Assert.assertNotNull(url, "Could not find resource " + url); + this.defaultConfigs = ConfigFactory.parseURL(url).resolve(); + } + + @Test + public void testBuildContainerCommand() throws Exception { + final double jvmMemoryXmxRatio = 0.7; + final int jvmMemoryOverheadMbs = 50; + final int resourceMemoryMB = 3072; + final int expectedJvmMemory = (int) (resourceMemoryMB * jvmMemoryXmxRatio) - jvmMemoryOverheadMbs; + + Config config = this.defaultConfigs.withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryXmxRatio)) + .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryOverheadMbs)); + + Resource resource = Resource.newInstance(resourceMemoryMB, 2); + + Container mockContainer = Mockito.mock(Container.class); + Mockito.when(mockContainer.getResource()).thenReturn(resource); + Mockito.when(mockContainer.getAllocationRequestId()).thenReturn(0L); + + YarnService yarnService = new YarnService( + config, + "testApplicationName", + "testApplicationId", + yarnConfiguration, + Mockito.mock(FileSystem.class), + eventBus + ); + + String command = yarnService.buildContainerCommand(mockContainer, "testHelixParticipantId", "testHelixInstanceTag"); + Assert.assertTrue(command.contains("-Xmx" + expectedJvmMemory + "M")); + } +} diff --git a/gobblin-temporal/src/test/resources/YarnServiceTest.conf b/gobblin-temporal/src/test/resources/YarnServiceTest.conf new file mode 100644 index 00000000000..da529e59c9d --- /dev/null +++ b/gobblin-temporal/src/test/resources/YarnServiceTest.conf @@ -0,0 +1,5 @@ +# Adding some default configs used while initializing YarnService for tests +gobblin.yarn.container.memory.mbs=1024 +gobblin.yarn.container.cores=1 +gobblin.yarn.container.affinity.enabled=false +gobblin.yarn.helix.instance.max.retries=1 \ No newline at end of file From 388383757987aaf73e2f4f3f51eb3ea4a0119684 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Fri, 29 Nov 2024 23:51:52 +0530 Subject: [PATCH 14/23] added unit test for baselineworkerprofilecreation --- .../temporal/yarn/YarnServiceTest.java | 29 +++++++++++++++++-- 1 file changed, 27 insertions(+), 2 deletions(-) diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java index e5c4f957815..03b3034215f 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java @@ -35,7 +35,8 @@ /** Tests for {@link YarnService}*/ public class YarnServiceTest { private Config defaultConfigs; - private YarnConfiguration yarnConfiguration = new YarnConfiguration(); + private final YarnConfiguration yarnConfiguration = new YarnConfiguration(); + private final FileSystem mockFileSystem = Mockito.mock(FileSystem.class); private final EventBus eventBus = new EventBus("TemporalYarnServiceTest"); @BeforeClass @@ -46,6 +47,30 @@ public void setup() { this.defaultConfigs = ConfigFactory.parseURL(url).resolve(); } + @Test + public void testBaselineWorkerProfileCreatedWithPassedConfigs() throws Exception { + final int containerMemoryMbs = 1500; + final int containerCores = 5; + final int numContainers = 4; + Config config = this.defaultConfigs + .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, ConfigValueFactory.fromAnyRef(containerMemoryMbs)) + .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, ConfigValueFactory.fromAnyRef(containerCores)) + .withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, ConfigValueFactory.fromAnyRef(numContainers)); + + YarnService yarnService = new YarnService( + config, + "testApplicationName", + "testApplicationId", + yarnConfiguration, + mockFileSystem, + eventBus + ); + + Assert.assertEquals(yarnService.baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), containerMemoryMbs); + Assert.assertEquals(yarnService.baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY), containerCores); + Assert.assertEquals(yarnService.baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY), numContainers); + } + @Test public void testBuildContainerCommand() throws Exception { final double jvmMemoryXmxRatio = 0.7; @@ -67,7 +92,7 @@ public void testBuildContainerCommand() throws Exception { "testApplicationName", "testApplicationId", yarnConfiguration, - Mockito.mock(FileSystem.class), + mockFileSystem, eventBus ); From c478eb53c343e93f4927c142061fc86bee55e666 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Sat, 30 Nov 2024 00:45:42 +0530 Subject: [PATCH 15/23] added unit tests for runnable --- ...tractDynamicScalingYarnServiceManager.java | 4 +- .../DynamicScalingYarnServiceManagerTest.java | 89 +++++++++++++++++++ 2 files changed, 92 insertions(+), 1 deletion(-) create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java index bb0853196c6..788101cce57 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java @@ -23,6 +23,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import org.apache.commons.collections.CollectionUtils; + import com.typesafe.config.Config; import com.google.common.util.concurrent.AbstractIdleService; @@ -103,7 +105,7 @@ static class GetScalingDirectivesRunnable implements Runnable { public void run() { try { List scalingDirectives = scalingDirectiveSource.getScalingDirectives(); - if (!scalingDirectives.isEmpty()) { + if (CollectionUtils.isNotEmpty(scalingDirectives)) { dynamicScalingYarnService.reviseWorkforcePlanAndRequestNewContainers(scalingDirectives); } } catch (IOException e) { diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java new file mode 100644 index 00000000000..2fc61d8d2b5 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java @@ -0,0 +1,89 @@ +package org.apache.gobblin.temporal.yarn; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.mockito.Mockito; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; +import org.apache.gobblin.temporal.loadgen.dynamic.DummyScalingDirectiveSource; + +/** Tests for {@link AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable}*/ +public class DynamicScalingYarnServiceManagerTest { + + private DynamicScalingYarnService mockDynamicScalingYarnService; + private ScalingDirectiveSource mockScalingDirectiveSource; + + @BeforeMethod + public void setup() { + mockDynamicScalingYarnService = Mockito.mock(DynamicScalingYarnService.class); + mockScalingDirectiveSource = Mockito.mock(ScalingDirectiveSource.class); + } + + @Test + public void testWhenScalingDirectivesIsNull() throws IOException { + Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(null); + Runnable runnable = new AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable( + mockDynamicScalingYarnService, mockScalingDirectiveSource); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + } + + @Test + public void testWhenScalingDirectivesIsEmpty() throws IOException { + Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(new ArrayList<>()); + Runnable runnable = new AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable( + mockDynamicScalingYarnService, mockScalingDirectiveSource); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + } + + /** Note : this test uses {@link org.apache.gobblin.temporal.loadgen.dynamic.DummyScalingDirectiveSource}*/ + @Test + public void testWithDummyScalingDirectiveSource() { + // DummyScalingDirectiveSource returns 2 scaling directives in first 3 invocations and after that it returns empty list + // so the total number of invocations after three invocations should always be 3 + Runnable runnable = new AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable( + mockDynamicScalingYarnService, new DummyScalingDirectiveSource()); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(1)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + } + + @Test + public void testWithRandomScalingDirectives() throws IOException { + ScalingDirective mockScalingDirective = Mockito.mock(ScalingDirective.class); + List mockedScalingDirectives = Arrays.asList(mockScalingDirective, mockScalingDirective); + Mockito.when(mockScalingDirectiveSource.getScalingDirectives()) + .thenReturn(new ArrayList<>()) + .thenReturn(mockedScalingDirectives) + .thenReturn(null) + .thenReturn(mockedScalingDirectives); + + Runnable runnable = new AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable( + mockDynamicScalingYarnService, mockScalingDirectiveSource); + + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(1)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(1)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + runnable.run(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + } + + +} From 15396fce877161d559cabb178ffed69ac016d870 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Sat, 30 Nov 2024 19:34:47 +0530 Subject: [PATCH 16/23] refactored tests --- ...tractDynamicScalingYarnServiceManager.java | 2 +- .../DynamicScalingYarnServiceManagerTest.java | 114 ++++++++++++------ .../temporal/yarn/YarnServiceTest.java | 16 ++- 3 files changed, 88 insertions(+), 44 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java index 788101cce57..5ff2d73ef3b 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java @@ -47,7 +47,7 @@ @Slf4j public abstract class AbstractDynamicScalingYarnServiceManager extends AbstractIdleService { - private final static String DYNAMIC_SCALING_POLLING_INTERVAL = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "polling.interval"; + protected final static String DYNAMIC_SCALING_POLLING_INTERVAL = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "polling.interval"; private final int DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS = 60; protected final Config config; private final DynamicScalingYarnService dynamicScalingYarnService; diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java index 2fc61d8d2b5..5908527a312 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java @@ -1,3 +1,20 @@ +/* + * 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.yarn; import java.io.IOException; @@ -5,85 +22,108 @@ import java.util.Arrays; import java.util.List; +import org.mockito.Mock; import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + import org.apache.gobblin.temporal.dynamic.ScalingDirective; import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; import org.apache.gobblin.temporal.loadgen.dynamic.DummyScalingDirectiveSource; +import static org.apache.gobblin.temporal.yarn.AbstractDynamicScalingYarnServiceManager.DYNAMIC_SCALING_POLLING_INTERVAL; + /** Tests for {@link AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable}*/ public class DynamicScalingYarnServiceManagerTest { - private DynamicScalingYarnService mockDynamicScalingYarnService; - private ScalingDirectiveSource mockScalingDirectiveSource; + @Mock private DynamicScalingYarnService mockDynamicScalingYarnService; + @Mock private ScalingDirectiveSource mockScalingDirectiveSource; + @Mock private GobblinTemporalApplicationMaster mockGobblinTemporalApplicationMaster; @BeforeMethod public void setup() { - mockDynamicScalingYarnService = Mockito.mock(DynamicScalingYarnService.class); - mockScalingDirectiveSource = Mockito.mock(ScalingDirectiveSource.class); + MockitoAnnotations.openMocks(this); + // Using 1 second as polling interval so that the test runs faster and + // GetScalingDirectivesRunnable.run() will be called equal to amount of sleep introduced between startUp + // and shutDown in seconds + Config config = ConfigFactory.empty().withValue(DYNAMIC_SCALING_POLLING_INTERVAL, ConfigValueFactory.fromAnyRef(1)); + Mockito.when(mockGobblinTemporalApplicationMaster.getConfig()).thenReturn(config); + Mockito.when(mockGobblinTemporalApplicationMaster.get_yarnService()).thenReturn(mockDynamicScalingYarnService); } @Test - public void testWhenScalingDirectivesIsNull() throws IOException { + public void testWhenScalingDirectivesIsNull() throws IOException, InterruptedException { Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(null); - Runnable runnable = new AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable( - mockDynamicScalingYarnService, mockScalingDirectiveSource); - runnable.run(); + TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( + mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); + testDynamicScalingYarnServiceManager.startUp(); + Thread.sleep(2000); + testDynamicScalingYarnServiceManager.shutDown(); Mockito.verify(mockDynamicScalingYarnService, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); } @Test - public void testWhenScalingDirectivesIsEmpty() throws IOException { + public void testWhenScalingDirectivesIsEmpty() throws IOException, InterruptedException { Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(new ArrayList<>()); - Runnable runnable = new AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable( - mockDynamicScalingYarnService, mockScalingDirectiveSource); - runnable.run(); + TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( + mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); + testDynamicScalingYarnServiceManager.startUp(); + Thread.sleep(2000); + testDynamicScalingYarnServiceManager.shutDown(); Mockito.verify(mockDynamicScalingYarnService, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); } /** Note : this test uses {@link org.apache.gobblin.temporal.loadgen.dynamic.DummyScalingDirectiveSource}*/ @Test - public void testWithDummyScalingDirectiveSource() { + public void testWithDummyScalingDirectiveSource() throws InterruptedException { // DummyScalingDirectiveSource returns 2 scaling directives in first 3 invocations and after that it returns empty list // so the total number of invocations after three invocations should always be 3 - Runnable runnable = new AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable( - mockDynamicScalingYarnService, new DummyScalingDirectiveSource()); - runnable.run(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(1)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - runnable.run(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - runnable.run(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - runnable.run(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - runnable.run(); + TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( + mockGobblinTemporalApplicationMaster, new DummyScalingDirectiveSource()); + testDynamicScalingYarnServiceManager.startUp(); + Thread.sleep(5000); // 5 seconds sleep so that GetScalingDirectivesRunnable.run() is called for 5 times + testDynamicScalingYarnServiceManager.shutDown(); Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); } @Test - public void testWithRandomScalingDirectives() throws IOException { + public void testWithRandomScalingDirectives() throws IOException, InterruptedException { ScalingDirective mockScalingDirective = Mockito.mock(ScalingDirective.class); List mockedScalingDirectives = Arrays.asList(mockScalingDirective, mockScalingDirective); Mockito.when(mockScalingDirectiveSource.getScalingDirectives()) .thenReturn(new ArrayList<>()) .thenReturn(mockedScalingDirectives) - .thenReturn(null) - .thenReturn(mockedScalingDirectives); - - Runnable runnable = new AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable( - mockDynamicScalingYarnService, mockScalingDirectiveSource); + .thenReturn(mockedScalingDirectives) + .thenReturn(null); - runnable.run(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - runnable.run(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(1)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - runnable.run(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(1)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - runnable.run(); + TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( + mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); + testDynamicScalingYarnServiceManager.startUp(); + Thread.sleep(5000); + testDynamicScalingYarnServiceManager.shutDown(); Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); } + /** Test implementation of {@link AbstractDynamicScalingYarnServiceManager} which returns passed + * {@link ScalingDirectiveSource} when {@link #createScalingDirectiveSource()} is called while initialising + * {@link AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable} + * */ + protected static class TestDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { + private final ScalingDirectiveSource _scalingDirectiveSource; + public TestDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster, ScalingDirectiveSource scalingDirectiveSource) { + super(appMaster); + this._scalingDirectiveSource = scalingDirectiveSource; + } + + @Override + protected ScalingDirectiveSource createScalingDirectiveSource() { + return this._scalingDirectiveSource; + } + } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java index 03b3034215f..4fb30c199bf 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java @@ -17,21 +17,25 @@ package org.apache.gobblin.temporal.yarn; -import com.google.common.eventbus.EventBus; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.typesafe.config.ConfigValueFactory; -import org.apache.hadoop.yarn.api.records.Container; import java.net.URL; -import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.conf.YarnConfiguration; + import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; +import com.google.common.eventbus.EventBus; + +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + /** Tests for {@link YarnService}*/ public class YarnServiceTest { private Config defaultConfigs; From e5119686a9d5229270d18b600e16a5d46593cf32 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Sun, 1 Dec 2024 21:49:44 +0530 Subject: [PATCH 17/23] minor comment update --- .../yarn/AbstractDynamicScalingYarnServiceManager.java | 7 ++++--- .../java/org/apache/gobblin/temporal/yarn/YarnService.java | 3 ++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java index 5ff2d73ef3b..2bfb2c952df 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java @@ -26,6 +26,7 @@ import org.apache.commons.collections.CollectionUtils; import com.typesafe.config.Config; +import com.google.common.base.Optional; import com.google.common.util.concurrent.AbstractIdleService; import lombok.AllArgsConstructor; @@ -64,8 +65,8 @@ public AbstractDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster throw new RuntimeException(errorMsg); } this.dynamicScalingExecutor = Executors.newSingleThreadScheduledExecutor( - ExecutorsUtils.newThreadFactory(com.google.common.base.Optional.of(log), - com.google.common.base.Optional.of("DynamicScalingExecutor"))); + ExecutorsUtils.newThreadFactory(Optional.of(log), + Optional.of("DynamicScalingExecutor"))); } @Override @@ -84,7 +85,7 @@ protected void startUp() { @Override protected void shutDown() { log.info("Stopping the " + this.getClass().getSimpleName()); - ExecutorsUtils.shutdownExecutorService(this.dynamicScalingExecutor, com.google.common.base.Optional.of(log)); + ExecutorsUtils.shutdownExecutorService(this.dynamicScalingExecutor, Optional.of(log)); } /** diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index 6fa64815f9b..a16787374f3 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -810,7 +810,8 @@ public void onContainersAllocated(List containers) { // Find matching requests and remove the request (YARN-660). We the scheduler are responsible // for cleaning up requests after allocation based on the design in the described ticket. // YARN does not have a delta request API and the requests are not cleaned up automatically. - // Try finding a match first with the host as the resource name then fall back to any resource match. + // Try finding a match first with requestAllocationId (which should always be the case) then fall back to + // finding a match with the host as the resource name which then will fall back to any resource match. // Also see YARN-1902. Container count will explode without this logic for removing container requests. Collection matchingRequestsByAllocationRequestId = amrmClientAsync.getMatchingRequests(container.getAllocationRequestId()); if (!matchingRequestsByAllocationRequestId.isEmpty()) { From 88cd73212612edbfbe89387dee12d94c9d887dd9 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Thu, 5 Dec 2024 16:18:56 +0530 Subject: [PATCH 18/23] addressed few comments --- .../temporal/dynamic/WorkerProfile.java | 7 +++ ...tractDynamicScalingYarnServiceManager.java | 9 +-- .../yarn/DynamicScalingYarnService.java | 25 ++++----- ...ourceDynamicScalingYarnServiceManager.java | 4 +- .../gobblin/temporal/yarn/YarnService.java | 42 +++++++------- .../dynamic/DummyScalingDirectiveSource.java | 21 ++----- ...DummyDynamicScalingYarnServiceManager.java | 8 +-- .../DynamicScalingYarnServiceManagerTest.java | 25 +++------ .../temporal/yarn/YarnServiceTest.java | 56 ++++++++++++++----- .../src/test/resources/YarnServiceTest.conf | 1 + 10 files changed, 105 insertions(+), 93 deletions(-) rename gobblin-temporal/src/{main/java/org/apache/gobblin/temporal/loadgen => test/java/org/apache/gobblin/temporal}/dynamic/DummyScalingDirectiveSource.java (80%) rename gobblin-temporal/src/{main/java/org/apache/gobblin/temporal/loadgen/dynamic => test/java/org/apache/gobblin/temporal/yarn}/DummyDynamicScalingYarnServiceManager.java (80%) 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 bf1f1d2e099..7e658b176a5 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 @@ -18,12 +18,19 @@ package org.apache.gobblin.temporal.dynamic; import com.typesafe.config.Config; +import lombok.AllArgsConstructor; import lombok.Data; /** A named worker {@link Config} */ @Data +@AllArgsConstructor public class WorkerProfile { private final String name; private final Config config; + + public WorkerProfile(Config config) { + this.name = WorkforceProfiles.BASELINE_NAME; + this.config = config; + } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java index 2bfb2c952df..4679ffe62b5 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java @@ -44,6 +44,8 @@ * * This is an abstract class that provides the basic functionality for managing dynamic scaling. Subclasses should implement * {@link #createScalingDirectiveSource()} to provide a {@link ScalingDirectiveSource} that will be used to get scaling directives. + * + * The actual implemented class needs to be passed as value of config {@link org.apache.gobblin.yarn.GobblinYarnConfigurationKeys#APP_MASTER_SERVICE_CLASSES} */ @Slf4j public abstract class AbstractDynamicScalingYarnServiceManager extends AbstractIdleService { @@ -60,7 +62,7 @@ public AbstractDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster this.dynamicScalingYarnService = (DynamicScalingYarnService) appMaster.get_yarnService(); } else { String errorMsg = "Failure while getting YarnService Instance from GobblinTemporalApplicationMaster::get_yarnService()" - + " YarnService is not an instance of DynamicScalingYarnService"; + + " YarnService {" + appMaster.get_yarnService().getClass().getSimpleName() + "} is not an instance of DynamicScalingYarnService"; log.error(errorMsg); throw new RuntimeException(errorMsg); } @@ -73,8 +75,7 @@ public AbstractDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster protected void startUp() { int scheduleInterval = ConfigUtils.getInt(this.config, DYNAMIC_SCALING_POLLING_INTERVAL, DEFAULT_DYNAMIC_SCALING_POLLING_INTERVAL_SECS); - log.info("Starting the " + this.getClass().getSimpleName()); - log.info("Scheduling the dynamic scaling task with an interval of {} seconds", scheduleInterval); + log.info("Starting the {} with re-scaling interval of {} seconds", this.getClass().getSimpleName(), scheduleInterval); this.dynamicScalingExecutor.scheduleAtFixedRate( new GetScalingDirectivesRunnable(this.dynamicScalingYarnService, createScalingDirectiveSource()), @@ -112,7 +113,7 @@ public void run() { } catch (IOException e) { log.error("Failed to get scaling directives", e); } catch (Throwable t) { - log.error("Suppressing error from GetScalingDirectivesRunnable.run()", t); + log.error("Unexpected error with dynamic scaling via directives", t); } } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index a2351eb1d97..46b4b707cf2 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -44,7 +44,7 @@ public class DynamicScalingYarnService extends YarnService { /** this holds the current count of containers requested for each worker profile */ - private final WorkforceStaffing workforceStaffing; + private final WorkforceStaffing actualWorkforceStaffing; /** this holds the current total workforce plan as per latest received scaling directives */ private final WorkforcePlan workforcePlan; @@ -52,10 +52,14 @@ public DynamicScalingYarnService(Config config, String applicationName, String a YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { super(config, applicationName, applicationId, yarnConfiguration, fs, eventBus); - int initialContainers = this.baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); + this.actualWorkforceStaffing = WorkforceStaffing.initialize(0); + this.workforcePlan = new WorkforcePlan(this.config, this.config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY)); + } - this.workforceStaffing = WorkforceStaffing.initialize(initialContainers); - this.workforcePlan = new WorkforcePlan(this.baselineWorkerProfile.getConfig(), initialContainers); + @Override + protected void requestInitialContainers() { + StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.actualWorkforceStaffing); + requestNewContainersForStaffingDeltas(deltas); } /** @@ -65,7 +69,7 @@ public DynamicScalingYarnService(Config config, String applicationName, String a */ public synchronized void reviseWorkforcePlanAndRequestNewContainers(List scalingDirectives) { this.workforcePlan.reviseWhenNewer(scalingDirectives); - StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.workforceStaffing); + StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.actualWorkforceStaffing); requestNewContainersForStaffingDeltas(deltas); } @@ -74,13 +78,13 @@ private synchronized void requestNewContainersForStaffingDeltas(StaffingDeltas d if (profileDelta.getDelta() > 0) { WorkerProfile workerProfile = profileDelta.getProfile(); String profileName = workerProfile.getName(); - int currNumContainers = this.workforceStaffing.getStaffing(profileName).orElse(0); + int currNumContainers = this.actualWorkforceStaffing.getStaffing(profileName).orElse(0); int delta = profileDelta.getDelta(); log.info("Requesting {} new containers for profile {} having currently {} containers", delta, profileName, currNumContainers); requestContainersForWorkerProfile(workerProfile, delta); // update our staffing after requesting new containers - this.workforceStaffing.reviseStaffing(profileName, currNumContainers + delta, System.currentTimeMillis()); + this.actualWorkforceStaffing.reviseStaffing(profileName, currNumContainers + delta, System.currentTimeMillis()); } else { // TODO: Decide how to handle negative deltas log.warn("Handling of Negative delta is not supported yet : Profile {} delta {} ", @@ -89,11 +93,4 @@ private synchronized void requestNewContainersForStaffingDeltas(StaffingDeltas d }); } - private synchronized void requestContainersForWorkerProfile(WorkerProfile workerProfile, int numContainers) { - int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); - int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); - long allocationRequestId = storeByUniqueAllocationRequestId(workerProfile); - requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores), Optional.of(allocationRequestId)); - } - } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java index 88130ce9cc2..dca5c59a072 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java @@ -29,8 +29,8 @@ * {@link FsScalingDirectiveSource} based implementation of {@link AbstractDynamicScalingYarnServiceManager}. */ public class FsSourceDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { - private final static String DYNAMIC_SCALING_DIRECTIVES_DIR = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "directives.dir"; - private final static String DYNAMIC_SCALING_ERRORS_DIR = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "errors.dir"; + public final static String DYNAMIC_SCALING_DIRECTIVES_DIR = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "directives.dir"; + public final static String DYNAMIC_SCALING_ERRORS_DIR = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "errors.dir"; private final FileSystem fs; public FsSourceDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index a16787374f3..3b7a85d9954 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -110,7 +110,6 @@ import org.apache.gobblin.yarn.event.ContainerShutdownRequest; import org.apache.gobblin.yarn.event.NewContainerRequest; import org.apache.gobblin.temporal.dynamic.WorkerProfile; -import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; /** * This class is responsible for all Yarn-related stuffs including ApplicationMaster registration, @@ -132,7 +131,7 @@ class YarnService extends AbstractIdleService { private final String appViewAcl; //Default helix instance tag derived from cluster level config private final String helixInstanceTags; - private final Config config; + protected final Config config; private final EventBus eventBus; @@ -195,7 +194,7 @@ class YarnService extends AbstractIdleService { private volatile boolean shutdownInProgress = false; private final boolean jarCacheEnabled; - protected final WorkerProfile baselineWorkerProfile; + private final WorkerProfile defaultWorkerProfile; private final AtomicLong allocationRequestIdGenerator = new AtomicLong(0L); private final ConcurrentMap workerProfileByAllocationRequestId = new ConcurrentHashMap<>(); @@ -257,11 +256,7 @@ public YarnService(Config config, String applicationName, String applicationId, this.jarCacheEnabled = ConfigUtils.getBoolean(this.config, GobblinYarnConfigurationKeys.JAR_CACHE_ENABLED, GobblinYarnConfigurationKeys.JAR_CACHE_ENABLED_DEFAULT); // Initialising this baseline worker profile to use as default worker profile in case allocation request id is not in map - this.baselineWorkerProfile = new WorkerProfile(WorkforceProfiles.BASELINE_NAME, this.config); - - // Putting baseline profile in the map for default allocation request id (0) - // adding it here to have deterministic allocation request id for baseline worker profile - storeByUniqueAllocationRequestId(this.baselineWorkerProfile); + this.defaultWorkerProfile = new WorkerProfile(this.config); } @SuppressWarnings("unused") @@ -327,8 +322,8 @@ protected synchronized void startUp() throws Exception { LOGGER.info("ApplicationMaster registration response: " + response); this.maxResourceCapacity = Optional.of(response.getMaximumResourceCapability()); - LOGGER.info("Requesting initial containers using baselineWorkerProfile"); - requestInitialContainers(this.baselineWorkerProfile); + LOGGER.info("Requesting initial containers"); + requestInitialContainers(); } @Override @@ -403,14 +398,19 @@ private EventSubmitter buildEventSubmitter() { .build(); } - /** Request Initial containers using baselineWorkerProfile */ - private void requestInitialContainers(WorkerProfile workerProfile) { + /** unless overridden to actually scale, "initial" containers may be the app's *only* containers! */ + protected void requestInitialContainers() { + WorkerProfile baselineWorkerProfile = new WorkerProfile(this.config); + int numContainers = baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); + LOGGER.info("Requesting {} initial (static) containers with baseline (only) profile, never to be re-scaled", numContainers); + requestContainersForWorkerProfile(baselineWorkerProfile, numContainers); + } + + protected synchronized void requestContainersForWorkerProfile(WorkerProfile workerProfile, int numContainers) { int containerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); int containerCores = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY); - int numContainers = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); - LOGGER.info("Requesting {} initial containers with default allocation id = 0", numContainers); - // Using 0 as allocation id for baseline worker profile - requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores), Optional.of(0L)); + long allocationRequestId = storeByUniqueAllocationRequestId(workerProfile); + requestContainers(numContainers, Resource.newInstance(containerMemoryMbs, containerCores), Optional.of(allocationRequestId)); } private void requestContainer(Optional preferredNode, Optional resourceOptional) { @@ -559,7 +559,7 @@ protected String buildContainerCommand(Container container, String helixParticip long allocationRequestId = container.getAllocationRequestId(); // Using getOrDefault for backward-compatibility with containers that don't have allocationRequestId set WorkerProfile workerProfile = this.workerProfileByAllocationRequestId.getOrDefault(allocationRequestId, - this.baselineWorkerProfile); + this.defaultWorkerProfile); Config workerProfileConfig = workerProfile.getConfig(); double workerJvmMemoryXmxRatio = ConfigUtils.getDouble(workerProfileConfig, @@ -816,18 +816,18 @@ public void onContainersAllocated(List containers) { Collection matchingRequestsByAllocationRequestId = amrmClientAsync.getMatchingRequests(container.getAllocationRequestId()); if (!matchingRequestsByAllocationRequestId.isEmpty()) { AMRMClient.ContainerRequest firstMatchingContainerRequest = matchingRequestsByAllocationRequestId.iterator().next(); - LOGGER.debug("Found matching requests {}, removing first matching request {}", + LOGGER.info("Found matching requests {}, removing first matching request {}", matchingRequestsByAllocationRequestId, firstMatchingContainerRequest); amrmClientAsync.removeContainerRequest(firstMatchingContainerRequest); } else { - LOGGER.debug("Matching request by allocation request id {} not found", container.getAllocationRequestId()); + LOGGER.info("Matching request by allocation request id {} not found", container.getAllocationRequestId()); List> matchingRequestsByHost = amrmClientAsync .getMatchingRequests(container.getPriority(), container.getNodeHttpAddress(), container.getResource()); if (matchingRequestsByHost.isEmpty()) { - LOGGER.debug("Matching request by host {} not found", container.getNodeHttpAddress()); + LOGGER.info("Matching request by host {} not found", container.getNodeHttpAddress()); matchingRequestsByHost = amrmClientAsync .getMatchingRequests(container.getPriority(), ResourceRequest.ANY, container.getResource()); @@ -835,7 +835,7 @@ public void onContainersAllocated(List containers) { if (!matchingRequestsByHost.isEmpty()) { AMRMClient.ContainerRequest firstMatchingContainerRequest = matchingRequestsByHost.get(0).iterator().next(); - LOGGER.debug("Found matching requests {}, removing first matching request {}", + LOGGER.info("Found matching requests {}, removing first matching request {}", matchingRequestsByAllocationRequestId, firstMatchingContainerRequest); amrmClientAsync.removeContainerRequest(firstMatchingContainerRequest); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/DummyScalingDirectiveSource.java similarity index 80% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java rename to gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/DummyScalingDirectiveSource.java index fc5bb50f5a6..6bdfe46276f 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyScalingDirectiveSource.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/dynamic/DummyScalingDirectiveSource.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.loadgen.dynamic; +package org.apache.gobblin.temporal.dynamic; import java.util.ArrayList; import java.util.Arrays; @@ -24,11 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; -import org.apache.gobblin.temporal.dynamic.ProfileDerivation; -import org.apache.gobblin.temporal.dynamic.ProfileOverlay; -import org.apache.gobblin.temporal.dynamic.ScalingDirective; -import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; -import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; /** @@ -52,29 +47,25 @@ public DummyScalingDirectiveSource() { @Override public List getScalingDirectives() { // Note - profile should exist already or is derived from other profile - if (this.numInvocations.get() == 0) { - this.numInvocations.getAndIncrement(); + int currNumInvocations = this.numInvocations.getAndIncrement(); + long currentTime = System.currentTimeMillis(); + if (currNumInvocations == 0) { // here we are returning two new profile with initial container counts and these should be launched - long currentTime = System.currentTimeMillis(); // both profiles should have different timestampEpochMillis so that both are processed otherwise // org.apache.gobblin.temporal.dynamic.WorkforcePlan$IllegalRevisionException$OutOfOrderDirective can occur return Arrays.asList( new ScalingDirective("firstProfile", 3, currentTime, this.derivedFromBaseline), new ScalingDirective("secondProfile", 2, currentTime + 1, this.derivedFromBaseline) ); - } else if (this.numInvocations.get() == 1) { - this.numInvocations.getAndIncrement(); + } else if (currNumInvocations == 1) { // here we are increasing containers to 5 for firstProfile and 3 for secondProfile so that 2 new extra containers // should be launched for firstProfile and 1 new extra container for secondProfile - long currentTime = System.currentTimeMillis(); return Arrays.asList( new ScalingDirective("firstProfile", 5, currentTime), new ScalingDirective("secondProfile", 3, currentTime + 1) ); - } else if (this.numInvocations.get() == 2) { - this.numInvocations.getAndIncrement(); + } else if (currNumInvocations == 2) { // the count is same as previous invocation so no new containers should be launched - long currentTime = System.currentTimeMillis(); return Arrays.asList( new ScalingDirective("firstProfile", 5, currentTime), new ScalingDirective("secondProfile", 3, currentTime + 1) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java similarity index 80% rename from gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyDynamicScalingYarnServiceManager.java rename to gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java index a4ec19b587f..b79f8089381 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/loadgen/dynamic/DummyDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.gobblin.temporal.loadgen.dynamic; +package org.apache.gobblin.temporal.yarn; +import org.apache.gobblin.temporal.dynamic.DummyScalingDirectiveSource; import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; -import org.apache.gobblin.temporal.yarn.GobblinTemporalApplicationMaster; -import org.apache.gobblin.temporal.yarn.AbstractDynamicScalingYarnServiceManager; /** * {@link DummyScalingDirectiveSource} based implementation of {@link AbstractDynamicScalingYarnServiceManager}. - * This class is meant to be used for testing purposes only. + * This class is meant to be used for integration testing purposes only. + * This is initialized using config {@link org.apache.gobblin.yarn.GobblinYarnConfigurationKeys#APP_MASTER_SERVICE_CLASSES} while testing */ public class DummyDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java index 5908527a312..dd4243d3fa5 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java @@ -34,11 +34,11 @@ import org.apache.gobblin.temporal.dynamic.ScalingDirective; import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; -import org.apache.gobblin.temporal.loadgen.dynamic.DummyScalingDirectiveSource; +import org.apache.gobblin.temporal.dynamic.DummyScalingDirectiveSource; import static org.apache.gobblin.temporal.yarn.AbstractDynamicScalingYarnServiceManager.DYNAMIC_SCALING_POLLING_INTERVAL; -/** Tests for {@link AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable}*/ +/** Tests for {@link AbstractDynamicScalingYarnServiceManager}*/ public class DynamicScalingYarnServiceManagerTest { @Mock private DynamicScalingYarnService mockDynamicScalingYarnService; @@ -57,28 +57,17 @@ public void setup() { } @Test - public void testWhenScalingDirectivesIsNull() throws IOException, InterruptedException { - Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(null); + public void testWhenScalingDirectivesIsNulOrEmpty() throws IOException, InterruptedException { + Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(null).thenReturn(new ArrayList<>()); TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); testDynamicScalingYarnServiceManager.startUp(); - Thread.sleep(2000); + Thread.sleep(3000); testDynamicScalingYarnServiceManager.shutDown(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(mockDynamicScalingYarnService, Mockito.never()).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); } - @Test - public void testWhenScalingDirectivesIsEmpty() throws IOException, InterruptedException { - Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(new ArrayList<>()); - TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( - mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); - testDynamicScalingYarnServiceManager.startUp(); - Thread.sleep(2000); - testDynamicScalingYarnServiceManager.shutDown(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - } - - /** Note : this test uses {@link org.apache.gobblin.temporal.loadgen.dynamic.DummyScalingDirectiveSource}*/ + /** Note : this test uses {@link DummyScalingDirectiveSource}*/ @Test public void testWithDummyScalingDirectiveSource() throws InterruptedException { // DummyScalingDirectiveSource returns 2 scaling directives in first 3 invocations and after that it returns empty list diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java index 4fb30c199bf..67a4b460bfb 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java @@ -17,13 +17,21 @@ package org.apache.gobblin.temporal.yarn; +import java.io.IOException; import java.net.URL; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import com.google.common.base.Optional; + +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.mockito.MockedStatic; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.BeforeClass; @@ -36,33 +44,48 @@ import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; -/** Tests for {@link YarnService}*/ +import static org.mockito.Mockito.*; + + +/** + * Tests for {@link YarnService} + * + * NOTE : This test is a partial clone of {@link org.apache.gobblin.yarn.YarnServiceTest} + * */ public class YarnServiceTest { private Config defaultConfigs; private final YarnConfiguration yarnConfiguration = new YarnConfiguration(); private final FileSystem mockFileSystem = Mockito.mock(FileSystem.class); private final EventBus eventBus = new EventBus("TemporalYarnServiceTest"); + AMRMClientAsync mockAMRMClient; + RegisterApplicationMasterResponse mockRegisterApplicationMasterResponse; @BeforeClass - public void setup() { + public void setup() throws IOException, YarnException { + mockAMRMClient = mock(AMRMClientAsync.class); + mockRegisterApplicationMasterResponse = mock(RegisterApplicationMasterResponse.class); + URL url = YarnServiceTest.class.getClassLoader() .getResource(YarnServiceTest.class.getSimpleName() + ".conf"); Assert.assertNotNull(url, "Could not find resource " + url); this.defaultConfigs = ConfigFactory.parseURL(url).resolve(); + + MockedStatic amrmClientAsyncMockStatic = mockStatic(AMRMClientAsync.class); + + amrmClientAsyncMockStatic.when(() -> AMRMClientAsync.createAMRMClientAsync(anyInt(), any(AMRMClientAsync.CallbackHandler.class))) + .thenReturn(mockAMRMClient); + doNothing().when(mockAMRMClient).init(any(YarnConfiguration.class)); + + when(mockAMRMClient.registerApplicationMaster(anyString(), anyInt(), anyString())) + .thenReturn(mockRegisterApplicationMasterResponse); + when(mockRegisterApplicationMasterResponse.getMaximumResourceCapability()) + .thenReturn(Mockito.mock(Resource.class)); } @Test public void testBaselineWorkerProfileCreatedWithPassedConfigs() throws Exception { - final int containerMemoryMbs = 1500; - final int containerCores = 5; - final int numContainers = 4; - Config config = this.defaultConfigs - .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, ConfigValueFactory.fromAnyRef(containerMemoryMbs)) - .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, ConfigValueFactory.fromAnyRef(containerCores)) - .withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, ConfigValueFactory.fromAnyRef(numContainers)); - YarnService yarnService = new YarnService( - config, + this.defaultConfigs, "testApplicationName", "testApplicationId", yarnConfiguration, @@ -70,9 +93,11 @@ public void testBaselineWorkerProfileCreatedWithPassedConfigs() throws Exception eventBus ); - Assert.assertEquals(yarnService.baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), containerMemoryMbs); - Assert.assertEquals(yarnService.baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY), containerCores); - Assert.assertEquals(yarnService.baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY), numContainers); + yarnService.startUp(); + + Mockito.verify(yarnService, Mockito.never()) + .requestContainers(1, Mockito.any(Resource.class), Mockito.any(Optional.class)); + } @Test @@ -82,7 +107,8 @@ public void testBuildContainerCommand() throws Exception { final int resourceMemoryMB = 3072; final int expectedJvmMemory = (int) (resourceMemoryMB * jvmMemoryXmxRatio) - jvmMemoryOverheadMbs; - Config config = this.defaultConfigs.withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryXmxRatio)) + Config config = this.defaultConfigs + .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryXmxRatio)) .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryOverheadMbs)); Resource resource = Resource.newInstance(resourceMemoryMB, 2); diff --git a/gobblin-temporal/src/test/resources/YarnServiceTest.conf b/gobblin-temporal/src/test/resources/YarnServiceTest.conf index da529e59c9d..0903ced00b1 100644 --- a/gobblin-temporal/src/test/resources/YarnServiceTest.conf +++ b/gobblin-temporal/src/test/resources/YarnServiceTest.conf @@ -1,4 +1,5 @@ # Adding some default configs used while initializing YarnService for tests +gobblin.yarn.initial.containers=0 gobblin.yarn.container.memory.mbs=1024 gobblin.yarn.container.cores=1 gobblin.yarn.container.affinity.enabled=false From b593ca53b27a9b5e73ce0b39c752a0b1aba7d4a3 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Fri, 6 Dec 2024 10:37:56 +0530 Subject: [PATCH 19/23] fixed yarn service test --- .../temporal/dynamic/WorkerProfile.java | 8 ++++-- ...tractDynamicScalingYarnServiceManager.java | 2 +- .../yarn/DynamicScalingYarnService.java | 12 ++++----- ...ourceDynamicScalingYarnServiceManager.java | 1 + .../gobblin/temporal/yarn/YarnService.java | 18 +++++++------ .../temporal/yarn/YarnServiceTest.java | 25 ++++++++----------- 6 files changed, 34 insertions(+), 32 deletions(-) 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 7e658b176a5..d67825cb226 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 @@ -29,8 +29,12 @@ public class WorkerProfile { private final String name; private final Config config; + /** + * Constructs a `WorkerProfile` with the baseline name and the specified configuration. + * + * @param config the configuration for the worker profile + */ public WorkerProfile(Config config) { - this.name = WorkforceProfiles.BASELINE_NAME; - this.config = config; + this(WorkforceProfiles.BASELINE_NAME, config); } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java index 4679ffe62b5..3022d5c9ecc 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/AbstractDynamicScalingYarnServiceManager.java @@ -62,7 +62,7 @@ public AbstractDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster this.dynamicScalingYarnService = (DynamicScalingYarnService) appMaster.get_yarnService(); } else { String errorMsg = "Failure while getting YarnService Instance from GobblinTemporalApplicationMaster::get_yarnService()" - + " YarnService {" + appMaster.get_yarnService().getClass().getSimpleName() + "} is not an instance of DynamicScalingYarnService"; + + " YarnService {" + appMaster.get_yarnService().getClass().getName() + "} is not an instance of DynamicScalingYarnService"; log.error(errorMsg); throw new RuntimeException(errorMsg); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 46b4b707cf2..38b83442f74 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -20,10 +20,8 @@ import java.util.List; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import com.google.common.base.Optional; import com.google.common.eventbus.EventBus; import com.typesafe.config.Config; @@ -43,7 +41,7 @@ @Slf4j public class DynamicScalingYarnService extends YarnService { - /** this holds the current count of containers requested for each worker profile */ + /** this holds the current count of containers already requested for each worker profile */ private final WorkforceStaffing actualWorkforceStaffing; /** this holds the current total workforce plan as per latest received scaling directives */ private final WorkforcePlan workforcePlan; @@ -57,7 +55,7 @@ public DynamicScalingYarnService(Config config, String applicationName, String a } @Override - protected void requestInitialContainers() { + protected synchronized void requestInitialContainers() { StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.actualWorkforceStaffing); requestNewContainersForStaffingDeltas(deltas); } @@ -75,7 +73,7 @@ public synchronized void reviseWorkforcePlanAndRequestNewContainers(List { - if (profileDelta.getDelta() > 0) { + if (profileDelta.getDelta() > 0) { // scale up! WorkerProfile workerProfile = profileDelta.getProfile(); String profileName = workerProfile.getName(); int currNumContainers = this.actualWorkforceStaffing.getStaffing(profileName).orElse(0); @@ -85,11 +83,11 @@ private synchronized void requestNewContainersForStaffingDeltas(StaffingDeltas d requestContainersForWorkerProfile(workerProfile, delta); // update our staffing after requesting new containers this.actualWorkforceStaffing.reviseStaffing(profileName, currNumContainers + delta, System.currentTimeMillis()); - } else { + } else if (profileDelta.getDelta() < 0) { // scale down! // TODO: Decide how to handle negative deltas log.warn("Handling of Negative delta is not supported yet : Profile {} delta {} ", profileDelta.getProfile().getName(), profileDelta.getDelta()); - } + } // else, already at staffing plan (or at least have requested, so in-progress) }); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java index dca5c59a072..f6b65bbd2db 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/FsSourceDynamicScalingYarnServiceManager.java @@ -29,6 +29,7 @@ * {@link FsScalingDirectiveSource} based implementation of {@link AbstractDynamicScalingYarnServiceManager}. */ public class FsSourceDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { + // TODO: replace fetching of these configs using a new method similar to JobStateUtils::getWorkDirRoot public final static String DYNAMIC_SCALING_DIRECTIVES_DIR = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "directives.dir"; public final static String DYNAMIC_SCALING_ERRORS_DIR = GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_PREFIX + "errors.dir"; private final FileSystem fs; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index 3b7a85d9954..858e1135ca9 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -195,7 +195,8 @@ class YarnService extends AbstractIdleService { private final boolean jarCacheEnabled; private final WorkerProfile defaultWorkerProfile; - private final AtomicLong allocationRequestIdGenerator = new AtomicLong(0L); + private final long DEFAULT_ALLOCATION_REQUEST_ID = 0L; + private final AtomicLong allocationRequestIdGenerator = new AtomicLong(DEFAULT_ALLOCATION_REQUEST_ID); private final ConcurrentMap workerProfileByAllocationRequestId = new ConcurrentHashMap<>(); public YarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, @@ -398,10 +399,10 @@ private EventSubmitter buildEventSubmitter() { .build(); } - /** unless overridden to actually scale, "initial" containers may be the app's *only* containers! */ - protected void requestInitialContainers() { + /** unless overridden to actually scale, "initial" containers will be the app's *only* containers! */ + protected synchronized void requestInitialContainers() { WorkerProfile baselineWorkerProfile = new WorkerProfile(this.config); - int numContainers = baselineWorkerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); + int numContainers = this.config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); LOGGER.info("Requesting {} initial (static) containers with baseline (only) profile, never to be re-scaled", numContainers); requestContainersForWorkerProfile(baselineWorkerProfile, numContainers); } @@ -449,7 +450,7 @@ private void requestContainer(Optional preferredNode, Resource resource, String[] preferredNodes = preferredNode.isPresent() ? new String[] {preferredNode.get()} : null; - long allocationRequestId = optAllocationRequestId.or(0L); + long allocationRequestId = optAllocationRequestId.or(DEFAULT_ALLOCATION_REQUEST_ID); this.amrmClientAsync.addContainerRequest( new AMRMClient.ContainerRequest(resource, preferredNodes, null, priority, allocationRequestId)); @@ -558,8 +559,11 @@ protected ByteBuffer getSecurityTokens() throws IOException { protected String buildContainerCommand(Container container, String helixParticipantId, String helixInstanceTag) { long allocationRequestId = container.getAllocationRequestId(); // Using getOrDefault for backward-compatibility with containers that don't have allocationRequestId set - WorkerProfile workerProfile = this.workerProfileByAllocationRequestId.getOrDefault(allocationRequestId, - this.defaultWorkerProfile); + WorkerProfile workerProfile = Optional.fromNullable(this.workerProfileByAllocationRequestId.get(allocationRequestId)) + .or(() -> { + LOGGER.warn("No Worker Profile found for {} ... falling back... ", allocationRequestId); + return this.workerProfileByAllocationRequestId.get(DEFAULT_ALLOCATION_REQUEST_ID); + }); Config workerProfileConfig = workerProfile.getConfig(); double workerJvmMemoryXmxRatio = ConfigUtils.getDouble(workerProfileConfig, diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java index 67a4b460bfb..907f0cd3238 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java @@ -25,7 +25,6 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; -import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; import com.google.common.base.Optional; @@ -83,21 +82,15 @@ public void setup() throws IOException, YarnException { } @Test - public void testBaselineWorkerProfileCreatedWithPassedConfigs() throws Exception { - YarnService yarnService = new YarnService( - this.defaultConfigs, - "testApplicationName", - "testApplicationId", - yarnConfiguration, - mockFileSystem, - eventBus - ); - + public void testYarnServiceStartupWithInitialContainers() throws Exception { + int expectedNumContainers = 0; + YarnService yarnService = new YarnService(this.defaultConfigs, "testApplicationName", "testApplicationId", yarnConfiguration, mockFileSystem, eventBus) { + @Override + protected void requestContainers(int numContainers, Resource resource, Optional allocationRequestId) { + Assert.assertEquals(numContainers, expectedNumContainers); + } + }; yarnService.startUp(); - - Mockito.verify(yarnService, Mockito.never()) - .requestContainers(1, Mockito.any(Resource.class), Mockito.any(Optional.class)); - } @Test @@ -126,6 +119,8 @@ public void testBuildContainerCommand() throws Exception { eventBus ); + yarnService.startUp(); + String command = yarnService.buildContainerCommand(mockContainer, "testHelixParticipantId", "testHelixInstanceTag"); Assert.assertTrue(command.contains("-Xmx" + expectedJvmMemory + "M")); } From fbd4890fd203c8d45d12f45f5ebb85934a2a64a7 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Fri, 6 Dec 2024 15:55:56 +0530 Subject: [PATCH 20/23] added DynamicScalingYarnServiceTest --- .../yarn/DynamicScalingYarnService.java | 4 ++ .../yarn/DynamicScalingYarnServiceTest.java | 66 +++++++++++++++++++ .../temporal/yarn/YarnServiceTest.java | 4 +- 3 files changed, 72 insertions(+), 2 deletions(-) create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 38b83442f74..09c86a121e5 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -19,6 +19,7 @@ import java.util.List; +import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -66,6 +67,9 @@ protected synchronized void requestInitialContainers() { * @param scalingDirectives the list of scaling directives */ public synchronized void reviseWorkforcePlanAndRequestNewContainers(List scalingDirectives) { + if (CollectionUtils.isEmpty(scalingDirectives)) { + return; + } this.workforcePlan.reviseWhenNewer(scalingDirectives); StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.actualWorkforceStaffing); requestNewContainersForStaffingDeltas(deltas); diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java new file mode 100644 index 00000000000..1c29479fef7 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java @@ -0,0 +1,66 @@ +/* + * 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.yarn; + +import java.net.URL; +import java.util.Collections; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import com.google.common.base.Optional; +import com.google.common.eventbus.EventBus; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; + +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; + +/** Tests for {@link DynamicScalingYarnService} */ +public class DynamicScalingYarnServiceTest { + private Config defaultConfigs; + private final YarnConfiguration yarnConfiguration = new YarnConfiguration(); + private final FileSystem mockFileSystem = Mockito.mock(FileSystem.class); + private final EventBus eventBus = new EventBus("TemporalDynamicScalingYarnServiceTest"); + + @BeforeClass + public void setup() { + URL url = DynamicScalingYarnServiceTest.class.getClassLoader() + .getResource(YarnServiceTest.class.getSimpleName() + ".conf"); // using same initial config as of YarnServiceTest + Assert.assertNotNull(url, "Could not find resource " + url); + this.defaultConfigs = ConfigFactory.parseURL(url).resolve(); + } + + @Test + public void testReviseWorkforcePlanAndRequestNewContainers() throws Exception { + int numNewContainers = 5; + DynamicScalingYarnService dynamicScalingYarnService = new DynamicScalingYarnService(this.defaultConfigs, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus) { + @Override + protected void requestContainers(int numContainers, Resource resource, Optional allocationRequestId) { + Assert.assertEquals(numContainers, numNewContainers); + } + }; + ScalingDirective baseScalingDirective = new ScalingDirective(WorkforceProfiles.BASELINE_NAME, numNewContainers, System.currentTimeMillis()); + dynamicScalingYarnService.reviseWorkforcePlanAndRequestNewContainers(Collections.singletonList(baseScalingDirective)); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java index 907f0cd3238..dd61e16e8fb 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java @@ -56,8 +56,8 @@ public class YarnServiceTest { private final YarnConfiguration yarnConfiguration = new YarnConfiguration(); private final FileSystem mockFileSystem = Mockito.mock(FileSystem.class); private final EventBus eventBus = new EventBus("TemporalYarnServiceTest"); - AMRMClientAsync mockAMRMClient; - RegisterApplicationMasterResponse mockRegisterApplicationMasterResponse; + private AMRMClientAsync mockAMRMClient; + private RegisterApplicationMasterResponse mockRegisterApplicationMasterResponse; @BeforeClass public void setup() throws IOException, YarnException { From ae1fb4c918b33bf8852ee10c6be49044a0ace3c7 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Fri, 6 Dec 2024 15:58:36 +0530 Subject: [PATCH 21/23] removed unused line --- .../java/org/apache/gobblin/temporal/yarn/YarnService.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index 858e1135ca9..9c0ee1efd8d 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -194,7 +194,6 @@ class YarnService extends AbstractIdleService { private volatile boolean shutdownInProgress = false; private final boolean jarCacheEnabled; - private final WorkerProfile defaultWorkerProfile; private final long DEFAULT_ALLOCATION_REQUEST_ID = 0L; private final AtomicLong allocationRequestIdGenerator = new AtomicLong(DEFAULT_ALLOCATION_REQUEST_ID); private final ConcurrentMap workerProfileByAllocationRequestId = new ConcurrentHashMap<>(); @@ -256,8 +255,6 @@ public YarnService(Config config, String applicationName, String applicationId, GobblinYarnConfigurationKeys.DEFAULT_GOBBLIN_YARN_CONTAINER_TIMEZONE); this.jarCacheEnabled = ConfigUtils.getBoolean(this.config, GobblinYarnConfigurationKeys.JAR_CACHE_ENABLED, GobblinYarnConfigurationKeys.JAR_CACHE_ENABLED_DEFAULT); - // Initialising this baseline worker profile to use as default worker profile in case allocation request id is not in map - this.defaultWorkerProfile = new WorkerProfile(this.config); } @SuppressWarnings("unused") From 45033253f855c99d5b7c58ee259deee16111a5dd Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Sat, 7 Dec 2024 12:18:27 +0530 Subject: [PATCH 22/23] use renderName while printing profile name --- .../gobblin/temporal/yarn/DynamicScalingYarnService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 09c86a121e5..0720017b852 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -32,6 +32,7 @@ import org.apache.gobblin.temporal.dynamic.StaffingDeltas; import org.apache.gobblin.temporal.dynamic.WorkerProfile; import org.apache.gobblin.temporal.dynamic.WorkforcePlan; +import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; import org.apache.gobblin.temporal.dynamic.WorkforceStaffing; import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; @@ -83,7 +84,7 @@ private synchronized void requestNewContainersForStaffingDeltas(StaffingDeltas d int currNumContainers = this.actualWorkforceStaffing.getStaffing(profileName).orElse(0); int delta = profileDelta.getDelta(); log.info("Requesting {} new containers for profile {} having currently {} containers", delta, - profileName, currNumContainers); + WorkforceProfiles.renderName(profileName), currNumContainers); requestContainersForWorkerProfile(workerProfile, delta); // update our staffing after requesting new containers this.actualWorkforceStaffing.reviseStaffing(profileName, currNumContainers + delta, System.currentTimeMillis()); From 2e78702a953a7fae7e00b62450ca20b1f16bfa04 Mon Sep 17 00:00:00 2001 From: Vivek Rai Date: Mon, 9 Dec 2024 15:46:27 +0530 Subject: [PATCH 23/23] corrected tests --- .../apache/gobblin/temporal/yarn/YarnService.java | 10 ++++++---- .../yarn/DynamicScalingYarnServiceTest.java | 12 +++++------- .../gobblin/temporal/yarn/YarnServiceTest.java | 15 +++++++-------- 3 files changed, 18 insertions(+), 19 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index 9c0ee1efd8d..ec4da215a63 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -194,7 +194,7 @@ class YarnService extends AbstractIdleService { private volatile boolean shutdownInProgress = false; private final boolean jarCacheEnabled; - private final long DEFAULT_ALLOCATION_REQUEST_ID = 0L; + private static final long DEFAULT_ALLOCATION_REQUEST_ID = 0L; private final AtomicLong allocationRequestIdGenerator = new AtomicLong(DEFAULT_ALLOCATION_REQUEST_ID); private final ConcurrentMap workerProfileByAllocationRequestId = new ConcurrentHashMap<>(); @@ -555,11 +555,13 @@ protected ByteBuffer getSecurityTokens() throws IOException { @VisibleForTesting protected String buildContainerCommand(Container container, String helixParticipantId, String helixInstanceTag) { long allocationRequestId = container.getAllocationRequestId(); - // Using getOrDefault for backward-compatibility with containers that don't have allocationRequestId set WorkerProfile workerProfile = Optional.fromNullable(this.workerProfileByAllocationRequestId.get(allocationRequestId)) .or(() -> { - LOGGER.warn("No Worker Profile found for {} ... falling back... ", allocationRequestId); - return this.workerProfileByAllocationRequestId.get(DEFAULT_ALLOCATION_REQUEST_ID); + LOGGER.warn("No Worker Profile found for {}, so falling back to default", allocationRequestId); + return this.workerProfileByAllocationRequestId.computeIfAbsent(DEFAULT_ALLOCATION_REQUEST_ID, k -> { + LOGGER.warn("WARNING: (LIKELY) UNEXPECTED CONCURRENCY: No Worker Profile even yet mapped to the default allocation request ID {} - creating one now", DEFAULT_ALLOCATION_REQUEST_ID); + return new WorkerProfile(this.config); + }); }); Config workerProfileConfig = workerProfile.getConfig(); diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java index 1c29479fef7..6c0946aabbe 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java @@ -54,13 +54,11 @@ public void setup() { @Test public void testReviseWorkforcePlanAndRequestNewContainers() throws Exception { int numNewContainers = 5; - DynamicScalingYarnService dynamicScalingYarnService = new DynamicScalingYarnService(this.defaultConfigs, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus) { - @Override - protected void requestContainers(int numContainers, Resource resource, Optional allocationRequestId) { - Assert.assertEquals(numContainers, numNewContainers); - } - }; + DynamicScalingYarnService dynamicScalingYarnService = new DynamicScalingYarnService(this.defaultConfigs, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); + DynamicScalingYarnService dynamicScalingYarnServiceSpy = Mockito.spy(dynamicScalingYarnService); + Mockito.doNothing().when(dynamicScalingYarnServiceSpy).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); ScalingDirective baseScalingDirective = new ScalingDirective(WorkforceProfiles.BASELINE_NAME, numNewContainers, System.currentTimeMillis()); - dynamicScalingYarnService.reviseWorkforcePlanAndRequestNewContainers(Collections.singletonList(baseScalingDirective)); + dynamicScalingYarnServiceSpy.reviseWorkforcePlanAndRequestNewContainers(Collections.singletonList(baseScalingDirective)); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(numNewContainers), Mockito.any(Resource.class), Mockito.any(Optional.class)); } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java index dd61e16e8fb..3c81316b85c 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java @@ -83,14 +83,13 @@ public void setup() throws IOException, YarnException { @Test public void testYarnServiceStartupWithInitialContainers() throws Exception { - int expectedNumContainers = 0; - YarnService yarnService = new YarnService(this.defaultConfigs, "testApplicationName", "testApplicationId", yarnConfiguration, mockFileSystem, eventBus) { - @Override - protected void requestContainers(int numContainers, Resource resource, Optional allocationRequestId) { - Assert.assertEquals(numContainers, expectedNumContainers); - } - }; - yarnService.startUp(); + int expectedNumContainers = 3; + Config config = this.defaultConfigs.withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, ConfigValueFactory.fromAnyRef(expectedNumContainers)); + YarnService yarnService = new YarnService(config, "testApplicationName", "testApplicationId", yarnConfiguration, mockFileSystem, eventBus); + YarnService yarnServiceSpy = Mockito.spy(yarnService); + Mockito.doNothing().when(yarnServiceSpy).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); + yarnServiceSpy.startUp(); + Mockito.verify(yarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(expectedNumContainers), Mockito.any(Resource.class), Mockito.any(Optional.class)); } @Test