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


##########
docs/content/docs/custom-resource/autoscaler.md:
##########
@@ -260,17 +260,46 @@ job.autoscaler.metrics.window : 3m
 > `ScalingReport` will show the recommended parallelism for each vertex.
 
 After the flink job starts, please start the StandaloneAutoscaler process by 
the
-following command.
+following command. Please download released autoscaler-standalone jar from 
+[here](https://repo.maven.apache.org/maven2/org/apache/flink/flink-autoscaler-standalone/)
 first.
 
 ```
 java -cp flink-autoscaler-standalone-{{< version >}}.jar \
 org.apache.flink.autoscaler.standalone.StandaloneAutoscalerEntrypoint \
---flinkClusterHost localhost \
---flinkClusterPort 8081
+--autoscaler.standalone.fetcher.flink-cluster.host localhost \
+--autoscaler.standalone.fetcher.flink-cluster.port 8081
 ```
 
-Updating the `flinkClusterHost` and `flinkClusterPort` based on your flink 
cluster.
-In general, the host and port are the same as Flink WebUI.
+Updating the `autoscaler.standalone.fetcher.flink-cluster.host` and 
`autoscaler.standalone.fetcher.flink-cluster.port` 
+based on your flink cluster. In general, the host and port are the same as 
Flink WebUI.
+
+### Using the JDBC Autoscaler State Store
+
+A driver dependency is required to connect to a specified database. Here are 
drivers currently supported, 
+please download JDBC driver and initialize database and table first.
+
+| Driver     | Group Id                   | Artifact Id            | JAR       
                                                                      | Schema  
                |
+|:-----------|:---------------------------|:-----------------------|:--------------------------------------------------------------------------------|-------------------------|
+| MySQL      | `mysql`                    | `mysql-connector-java` | 
[Download](https://repo.maven.apache.org/maven2/mysql/mysql-connector-java/)    
| [Table 
DDL](https://github.com/apache/flink-kubernetes-operator/blob/main/flink-autoscaler-plugin-jdbc/src/main/resources/schema/mysql_schema.sql)
     |
+| PostgreSQL | `org.postgresql`           | `postgresql`           | 
[Download](https://jdbc.postgresql.org/download/)                               
| [Table 
DDL](https://github.com/apache/flink-kubernetes-operator/blob/main/flink-autoscaler-plugin-jdbc/src/main/resources/schema/postgres_schema.sql)
  |
+| Derby      | `org.apache.derby`         | `derby`                | 
[Download](http://db.apache.org/derby/derby_downloads.html)                     
| [Table 
DDL](https://github.com/apache/flink-kubernetes-operator/blob/main/flink-autoscaler-plugin-jdbc/src/main/resources/schema/derby_schema.sql)
     |
+
+```
+JDBC_DRIVER_JAR=./mysql-connector-java-8.0.30.jar
+# export the password of jdbc state store
+export STATE_STORE_JDBC_PWD=123456
+
+java -cp flink-autoscaler-standalone-{{< version >}}.jar:${JDBC_DRIVER_JAR} \
+org.apache.flink.autoscaler.standalone.StandaloneAutoscalerEntrypoint \
+--autoscaler.standalone.fetcher.flink-cluster.host localhost \
+--autoscaler.standalone.fetcher.flink-cluster.port 8081 \
+--autoscaler.standalone.state-store.type jdbc \
+--autoscaler.standalone.state-store.jdbc.url 
jdbc:mysql://localhost:3306/flink_autoscaler \
+--autoscaler.standalone.state-store.jdbc.username root

Review Comment:
   should we also state the password-env-variable config here?



##########
flink-autoscaler-plugin-jdbc/src/main/java/org/apache/flink/autoscaler/jdbc/state/JdbcAutoScalerStateStore.java:
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.jdbc.state;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.autoscaler.JobAutoScalerContext;
+import org.apache.flink.autoscaler.ScalingSummary;
+import org.apache.flink.autoscaler.ScalingTracking;
+import org.apache.flink.autoscaler.metrics.CollectedMetrics;
+import org.apache.flink.autoscaler.state.AutoScalerStateStore;
+import org.apache.flink.autoscaler.utils.AutoScalerSerDeModule;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+import com.fasterxml.jackson.core.JacksonException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+import static 
org.apache.flink.autoscaler.jdbc.state.StateType.COLLECTED_METRICS;
+import static 
org.apache.flink.autoscaler.jdbc.state.StateType.PARALLELISM_OVERRIDES;
+import static org.apache.flink.autoscaler.jdbc.state.StateType.SCALING_HISTORY;
+import static 
org.apache.flink.autoscaler.jdbc.state.StateType.SCALING_TRACKING;
+
+/**
+ * The state store which persists its state in JDBC related database.
+ *
+ * @param <KEY> The job key.
+ * @param <Context> The job autoscaler context.
+ */
+@Experimental
+public class JdbcAutoScalerStateStore<KEY, Context extends 
JobAutoScalerContext<KEY>>
+        implements AutoScalerStateStore<KEY, Context> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(JdbcAutoScalerStateStore.class);
+
+    private final JdbcStateStore jdbcStateStore;
+
+    protected static final ObjectMapper YAML_MAPPER =
+            new ObjectMapper()
+                    .registerModule(new JavaTimeModule())
+                    .registerModule(new AutoScalerSerDeModule());
+
+    public JdbcAutoScalerStateStore(JdbcStateStore jdbcStateStore) {
+        this.jdbcStateStore = jdbcStateStore;
+    }
+
+    @Override
+    public void storeScalingHistory(
+            Context jobContext, Map<JobVertexID, SortedMap<Instant, 
ScalingSummary>> scalingHistory)
+            throws Exception {
+        jdbcStateStore.putSerializedState(
+                getSerializeKey(jobContext),
+                SCALING_HISTORY,
+                serializeScalingHistory(scalingHistory));
+    }
+
+    @Nonnull
+    @Override
+    public Map<JobVertexID, SortedMap<Instant, ScalingSummary>> 
getScalingHistory(
+            Context jobContext) {
+        Optional<String> serializedScalingHistory =
+                jdbcStateStore.getSerializedState(getSerializeKey(jobContext), 
SCALING_HISTORY);
+        if (serializedScalingHistory.isEmpty()) {
+            return new HashMap<>();
+        }
+        try {
+            return deserializeScalingHistory(serializedScalingHistory.get());
+        } catch (JacksonException e) {
+            LOG.error(
+                    "Could not deserialize scaling history, possibly the 
format changed. Discarding...",
+                    e);
+            jdbcStateStore.removeSerializedState(getSerializeKey(jobContext), 
SCALING_HISTORY);
+            return new HashMap<>();
+        }
+    }
+
+    @Override
+    public void removeScalingHistory(Context jobContext) {
+        jdbcStateStore.removeSerializedState(getSerializeKey(jobContext), 
SCALING_HISTORY);
+    }
+
+    @Override
+    public void storeScalingTracking(Context jobContext, ScalingTracking 
scalingTrack)
+            throws Exception {
+        jdbcStateStore.putSerializedState(
+                getSerializeKey(jobContext),
+                SCALING_TRACKING,
+                serializeScalingTracking(scalingTrack));
+    }
+
+    @Override
+    public ScalingTracking getScalingTracking(Context jobContext) {
+        Optional<String> serializedRescalingHistory =
+                jdbcStateStore.getSerializedState(getSerializeKey(jobContext), 
SCALING_TRACKING);
+        if (serializedRescalingHistory.isEmpty()) {
+            return new ScalingTracking();
+        }
+        try {
+            return 
deserializeScalingTracking(serializedRescalingHistory.get());
+        } catch (JacksonException e) {
+            LOG.error(
+                    "Could not deserialize rescaling history, possibly the 
format changed. Discarding...",
+                    e);
+            jdbcStateStore.removeSerializedState(getSerializeKey(jobContext), 
SCALING_TRACKING);
+            return new ScalingTracking();
+        }
+    }
+
+    @Override
+    public void storeCollectedMetrics(
+            Context jobContext, SortedMap<Instant, CollectedMetrics> metrics) 
throws Exception {
+        jdbcStateStore.putSerializedState(
+                getSerializeKey(jobContext), COLLECTED_METRICS, 
serializeEvaluatedMetrics(metrics));
+    }
+
+    @Nonnull
+    @Override
+    public SortedMap<Instant, CollectedMetrics> getCollectedMetrics(Context 
jobContext) {
+        Optional<String> serializedEvaluatedMetricsOpt =
+                jdbcStateStore.getSerializedState(getSerializeKey(jobContext), 
COLLECTED_METRICS);
+        if (serializedEvaluatedMetricsOpt.isEmpty()) {
+            return new TreeMap<>();
+        }
+        try {
+            return 
deserializeEvaluatedMetrics(serializedEvaluatedMetricsOpt.get());
+        } catch (JacksonException e) {
+            LOG.error(
+                    "Could not deserialize metric history, possibly the format 
changed. Discarding...",
+                    e);
+            jdbcStateStore.removeSerializedState(getSerializeKey(jobContext), 
COLLECTED_METRICS);
+            return new TreeMap<>();
+        }
+    }
+
+    @Override
+    public void removeCollectedMetrics(Context jobContext) {
+        jdbcStateStore.removeSerializedState(getSerializeKey(jobContext), 
COLLECTED_METRICS);
+    }
+
+    @Override
+    public void storeParallelismOverrides(
+            Context jobContext, Map<String, String> parallelismOverrides) {
+        jdbcStateStore.putSerializedState(
+                getSerializeKey(jobContext),
+                PARALLELISM_OVERRIDES,
+                serializeParallelismOverrides(parallelismOverrides));
+    }
+
+    @Nonnull
+    @Override
+    public Map<String, String> getParallelismOverrides(Context jobContext) {
+        return jdbcStateStore
+                .getSerializedState(getSerializeKey(jobContext), 
PARALLELISM_OVERRIDES)
+                .map(JdbcAutoScalerStateStore::deserializeParallelismOverrides)
+                .orElse(new HashMap<>());
+    }
+
+    @Override
+    public void removeParallelismOverrides(Context jobContext) {
+        jdbcStateStore.removeSerializedState(getSerializeKey(jobContext), 
PARALLELISM_OVERRIDES);
+    }
+
+    @Override
+    public void clearAll(Context jobContext) {
+        jdbcStateStore.clearAll(getSerializeKey(jobContext));
+    }
+
+    @Override
+    public void flush(Context jobContext) throws Exception {
+        jdbcStateStore.flush(getSerializeKey(jobContext));
+    }
+
+    @Override
+    public void removeInfoFromCache(KEY jobKey) {
+        jdbcStateStore.removeInfoFromCache(getSerializeKey(jobKey));
+    }
+
+    private String getSerializeKey(Context jobContext) {
+        return getSerializeKey(jobContext.getJobKey());
+    }
+
+    private String getSerializeKey(KEY jobKey) {
+        return jobKey.toString();
+    }
+
+    // The serialization and deserialization are similar to 
KubernetesAutoScalerStateStore
+    protected static String serializeScalingHistory(
+            Map<JobVertexID, SortedMap<Instant, ScalingSummary>> 
scalingHistory) throws Exception {
+        return compress(YAML_MAPPER.writeValueAsString(scalingHistory));
+    }
+
+    private static Map<JobVertexID, SortedMap<Instant, ScalingSummary>> 
deserializeScalingHistory(
+            String scalingHistory) throws JacksonException {
+        return YAML_MAPPER.readValue(decompress(scalingHistory), new 
TypeReference<>() {});
+    }
+
+    protected static String serializeScalingTracking(ScalingTracking 
scalingTracking)
+            throws Exception {
+        return compress(YAML_MAPPER.writeValueAsString(scalingTracking));
+    }
+
+    private static ScalingTracking deserializeScalingTracking(String 
scalingTracking)
+            throws JacksonException {
+        return YAML_MAPPER.readValue(decompress(scalingTracking), new 
TypeReference<>() {});
+    }
+
+    @VisibleForTesting
+    protected static String serializeEvaluatedMetrics(
+            SortedMap<Instant, CollectedMetrics> evaluatedMetrics) throws 
Exception {
+        return compress(YAML_MAPPER.writeValueAsString(evaluatedMetrics));
+    }
+
+    private static SortedMap<Instant, CollectedMetrics> 
deserializeEvaluatedMetrics(
+            String evaluatedMetrics) throws JacksonException {
+        return YAML_MAPPER.readValue(decompress(evaluatedMetrics), new 
TypeReference<>() {});
+    }
+
+    private static String serializeParallelismOverrides(Map<String, String> 
overrides) {
+        return ConfigurationUtils.convertValue(overrides, String.class);
+    }
+
+    private static Map<String, String> deserializeParallelismOverrides(String 
overrides) {
+        return ConfigurationUtils.convertValue(overrides, Map.class);
+    }
+
+    private static String compress(String original) throws IOException {

Review Comment:
   Is there really a reason to compress the data in a DB? For configmaps this 
was done due to the very constrained size. But compressing makes it much harder 
to read for users



-- 
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