gyfora commented on code in PR #698: URL: https://github.com/apache/flink-kubernetes-operator/pull/698#discussion_r1377406492
########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/realizer/RescaleApiScalingRealizer.java: ########## @@ -0,0 +1,153 @@ +/* + * 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.flink.autoscaler.realizer; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.autoscaler.JobAutoScalerContext; +import org.apache.flink.autoscaler.config.AutoScalerOptions; +import org.apache.flink.autoscaler.event.AutoScalerEventHandler; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.jobgraph.JobResourceRequirements; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.JobVertexResourceRequirements; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobMessageParameters; +import org.apache.flink.runtime.rest.messages.job.JobResourceRequirementsBody; +import org.apache.flink.runtime.rest.messages.job.JobResourceRequirementsHeaders; +import org.apache.flink.runtime.rest.messages.job.JobResourcesRequirementsUpdateHeaders; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** Rescaling job based on rescale api. */ +public class RescaleApiScalingRealizer<KEY, Context extends JobAutoScalerContext<KEY>> + implements ScalingRealizer<KEY, Context> { + + private static final Logger LOG = LoggerFactory.getLogger(RescaleApiScalingRealizer.class); + + private final AutoScalerEventHandler<KEY, JobAutoScalerContext<KEY>> eventHandler; + + public RescaleApiScalingRealizer( + AutoScalerEventHandler<KEY, JobAutoScalerContext<KEY>> eventHandler) { + this.eventHandler = eventHandler; + } + + @Override + public void realize(Context context, Map<String, String> parallelismOverrides) { + Configuration conf = context.getConfiguration(); + if (!conf.get(JobManagerOptions.SCHEDULER) + .equals(JobManagerOptions.SchedulerType.Adaptive)) { + LOG.debug("In-place rescaling is only available with the adaptive scheduler"); + return; + } + + JobStatus jobStatus = context.getJobStatus(); + JobID jobID = context.getJobID(); + if (jobID == null + || jobStatus == null + || jobStatus.isGloballyTerminalState() + || JobStatus.RECONCILING == jobStatus) { + LOG.info("Job in terminal or reconciling state cannot be scaled in-place"); + return; + } Review Comment: Why not simply check for "RUNNING" state? ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/realizer/RescaleApiScalingRealizer.java: ########## @@ -0,0 +1,153 @@ +/* + * 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.flink.autoscaler.realizer; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.autoscaler.JobAutoScalerContext; +import org.apache.flink.autoscaler.config.AutoScalerOptions; +import org.apache.flink.autoscaler.event.AutoScalerEventHandler; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.jobgraph.JobResourceRequirements; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.JobVertexResourceRequirements; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobMessageParameters; +import org.apache.flink.runtime.rest.messages.job.JobResourceRequirementsBody; +import org.apache.flink.runtime.rest.messages.job.JobResourceRequirementsHeaders; +import org.apache.flink.runtime.rest.messages.job.JobResourcesRequirementsUpdateHeaders; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** Rescaling job based on rescale api. */ +public class RescaleApiScalingRealizer<KEY, Context extends JobAutoScalerContext<KEY>> Review Comment: Please put a note that this is based on code copied from the operator , there is a lot of duplication, so there should be some justification for that. ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/standalone/flinkcluster/FlinkClusterJobListFetcher.java: ########## @@ -0,0 +1,109 @@ +/* + * 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.flink.autoscaler.standalone.flinkcluster; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.autoscaler.JobAutoScalerContext; +import org.apache.flink.autoscaler.standalone.JobListFetcher; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneClientHAServices; +import org.apache.flink.runtime.rest.messages.ConfigurationInfo; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobMessageParameters; +import org.apache.flink.runtime.rest.messages.job.JobManagerJobConfigurationHeaders; + +import java.time.Duration; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** Fetch JobAutoScalerContext based on flink cluster. */ +public class FlinkClusterJobListFetcher implements JobListFetcher<JobID> { + + private final String restServerAddress; + private final Duration restClientTimeout; + + public FlinkClusterJobListFetcher(String host, int port, Duration restClientTimeout) { + this.restServerAddress = String.format("http://%s:%s", host, port); + this.restClientTimeout = restClientTimeout; + } + + @Override + public List<JobAutoScalerContext<JobID>> fetch(MetricGroup metricGroup) throws Exception { Review Comment: I don't think `metricGroup` should be part of the method argument. The `FlinkClusterJobListFetcher` could under the hood simply pass the unregistered metric group. ########## flink-autoscaler/src/main/java/org/apache/flink/autoscaler/standalone/StandaloneAutoscalerEntrypoint.java: ########## @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.autoscaler.standalone; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.autoscaler.JobAutoScaler; +import org.apache.flink.autoscaler.JobAutoScalerContext; +import org.apache.flink.autoscaler.JobAutoScalerImpl; +import org.apache.flink.autoscaler.RestApiMetricsCollector; +import org.apache.flink.autoscaler.ScalingExecutor; +import org.apache.flink.autoscaler.ScalingMetricEvaluator; +import org.apache.flink.autoscaler.event.AutoScalerEventHandler; +import org.apache.flink.autoscaler.event.LoggableEventHandler; +import org.apache.flink.autoscaler.realizer.RescaleApiScalingRealizer; +import org.apache.flink.autoscaler.standalone.flinkcluster.FlinkClusterJobListFetcherFactory; +import org.apache.flink.autoscaler.state.AutoScalerStateStore; +import org.apache.flink.autoscaler.state.InMemoryAutoScalerStateStore; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.util.TimeUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.time.Instant; +import java.util.List; + +import static org.apache.flink.autoscaler.config.AutoScalerOptions.FLINK_CLIENT_TIMEOUT; + +/** The entrypoint of the standalone autoscaler. */ +@Experimental +public class StandaloneAutoscalerEntrypoint { + + private static final Logger LOG = LoggerFactory.getLogger(StandaloneAutoscalerEntrypoint.class); + + public static final String SCALING_INTERVAL = "scalingInterval"; + private static final Duration DEFAULT_SCALING_INTERVAL = Duration.ofSeconds(10); + + public static final String JOB_LIST_FETCHER_CLASS_NAME = "jobListFetcherClassName"; + private static final String DEFAULT_JOB_LIST_FETCHER_CLASS_NAME = + FlinkClusterJobListFetcherFactory.class.getName(); + + // This timeout option is used before the job config is got, such as: listJobs, get + // Configuration, etc. + public static final String REST_CLIENT_TIMEOUT = "restClientTimeout"; + + public static <KEY> void main(String[] args) throws Exception { + ParameterTool parameters = ParameterTool.fromArgs(args); + + Duration scalingInterval = DEFAULT_SCALING_INTERVAL; + if (parameters.get(SCALING_INTERVAL) != null) { + scalingInterval = TimeUtils.parseDuration(parameters.get(SCALING_INTERVAL)); + } + + Duration restClientTimeout = FLINK_CLIENT_TIMEOUT.defaultValue(); + if (parameters.get(REST_CLIENT_TIMEOUT) != null) { + restClientTimeout = TimeUtils.parseDuration(parameters.get(REST_CLIENT_TIMEOUT)); + } + + // Initialize JobListFetcher and JobAutoScaler. + JobListFetcherFactory<KEY> jobListFetcherFactory = getJobListFetcherFactory(parameters); + JobListFetcher<KEY> jobListFetcher = + jobListFetcherFactory.create(parameters, restClientTimeout); + + JobAutoScaler<KEY, JobAutoScalerContext<KEY>> autoScaler = createJobAutoscaler(); + + // Start control loop + Instant expectedNextSchedulerTime = Instant.now().plusMillis(scalingInterval.toMillis()); + while (true) { + LOG.info("Standalone autoscaler starts scaling."); + scaling(jobListFetcher, autoScaler); + + waitForNextSchedule(expectedNextSchedulerTime); + expectedNextSchedulerTime = Instant.now().plusMillis(scalingInterval.toMillis()); + } + } + + private static void waitForNextSchedule(Instant expectedNextSchedulerTime) + throws InterruptedException { + if (expectedNextSchedulerTime.isBefore(Instant.now())) { + return; + } + Duration sleepTime = Duration.between(Instant.now(), expectedNextSchedulerTime); + Thread.sleep(sleepTime.toMillis()); + } Review Comment: Can we use a ScheduledExecutorService for that to avoid sleeping etc? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
