gyfora commented on code in PR #677:
URL: 
https://github.com/apache/flink-kubernetes-operator/pull/677#discussion_r1343674332


##########
flink-autoscaler/src/main/java/org/apache/flink/autoscaler/DefaultJobAutoScalerContext.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.client.program.rest.RestClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.function.SupplierWithException;
+
+/** The default job autoscaler context, the jobKey is JobID. */
+public class DefaultJobAutoScalerContext extends 
AbstractJobAutoScalerContext<JobID> {

Review Comment:
   Let's simply delete the class and simply have a non-abstract 
`JobAutoscalerContext<JobId>`



##########
flink-autoscaler/src/main/java/org/apache/flink/autoscaler/AbstractJobAutoScalerContext.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.client.program.rest.RestClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.function.SupplierWithException;
+
+/**
+ * The abstract job autoscaler context.
+ *
+ * @param <KEY> The job key.
+ */
+public abstract class AbstractJobAutoScalerContext<KEY> implements 
JobAutoScalerContext<KEY> {

Review Comment:
   I would simply rename this as `JobAutoScalerContext<KEY>`, delete 
constructor , make all fields simply final and put `@Value` on it, to generate 
all the getters, toString etc.
   
   We don't need to have interfaces for everything, too much boilerplate



##########
flink-autoscaler/src/main/java/org/apache/flink/autoscaler/JobAutoScalerContext.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.flink.autoscaler;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.client.program.rest.RestClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.MetricGroup;
+
+/**
+ * The job autoscaler context, it includes all details related to the current 
job.
+ *
+ * @param <KEY> The job key.
+ */
+@Experimental
+public interface JobAutoScalerContext<KEY> {

Review Comment:
   Let's delete the interface for now (see my previous comment) we can re-add 
this later if necessary but I doubt it.



##########
flink-autoscaler/src/main/java/org/apache/flink/autoscaler/state/HeapedAutoScalerStateStore.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.state;
+
+import org.apache.flink.autoscaler.JobAutoScalerContext;
+import org.apache.flink.autoscaler.ScalingSummary;
+import org.apache.flink.autoscaler.metrics.CollectedMetrics;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import java.time.Instant;
+import java.util.Map;
+import java.util.Optional;
+import java.util.SortedMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The state store based on the Java Heap.
+ *
+ * @param <KEY>
+ * @param <Context>
+ */
+public class HeapedAutoScalerStateStore<KEY, Context extends 
JobAutoScalerContext<KEY>>

Review Comment:
   Let's rename this `HeapedAutoScalerStateStore` -> 
`InMemoryAutoScalerStateStore`



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to