zhuzhurk commented on code in PR #27388:
URL: https://github.com/apache/flink/pull/27388#discussion_r2670761479


##########
flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java:
##########
@@ -362,6 +374,62 @@ public enum JobStoreType {
         Memory
     }
 
+    /** The job store cache size in bytes which is used to keep completed jobs 
in memory. */
+    @Documentation.Section(Documentation.Sections.ALL_JOB_MANAGER)
+    public static final ConfigOption<Long> 
ARCHIVED_APPLICATION_STORE_CACHE_SIZE =
+            key("archived-application-store.cache-size")

Review Comment:
   Maybe name it as `completed-application-store` to make it more intuitive to 
users?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/util/ArchivedApplicationStoreUtils.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtime.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.dispatcher.ArchivedApplicationStore;
+
+import java.time.Duration;
+import java.util.Optional;
+
+/** Utilities for configuring of {@link ArchivedApplicationStore}. */
+public final class ArchivedApplicationStoreUtils {
+
+    public static JobManagerOptions.ArchivedApplicationStoreType getType(

Review Comment:
   We can simply use `ConfigOption#withDeprecatedKeys()` for config fallback.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedApplicationStoreTest.java:
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.runtime.dispatcher;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.runtime.application.ArchivedApplication;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetails;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.messages.webmonitor.JobsOverview;
+import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
+import org.apache.flink.runtime.util.ManualTicker;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter;
+
+import org.apache.flink.shaded.guava33.com.google.common.base.Ticker;
+import org.apache.flink.shaded.guava33.com.google.common.cache.LoadingCache;
+
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.runtime.dispatcher.ArchivedApplicationStoreTestUtils.createFileArchivedApplicationStore;
+import static 
org.apache.flink.runtime.dispatcher.ArchivedApplicationStoreTestUtils.generateApplicationDetails;
+import static 
org.apache.flink.runtime.dispatcher.ArchivedApplicationStoreTestUtils.generateJobDetails;
+import static 
org.apache.flink.runtime.dispatcher.ArchivedApplicationStoreTestUtils.generateTerminalArchivedApplications;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** Tests for the {@link FileArchivedApplicationStore}. */
+class FileArchivedApplicationStoreTest extends TestLogger {

Review Comment:
   Maybe let it extend `FileArchivedApplicationStoreTest` to reuse most of the 
tests?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/application/AbstractApplication.java:
##########
@@ -63,6 +65,8 @@ public abstract class AbstractApplication implements 
Serializable {
 
     private final Set<JobID> jobs = new HashSet<>();
 
+    private final List<ApplicationStatusListener> statusListeners = new 
ArrayList<>();

Review Comment:
   It should be `transient` and comments are needed to explain it.



##########
flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java:
##########
@@ -362,6 +374,62 @@ public enum JobStoreType {
         Memory
     }
 
+    /** The job store cache size in bytes which is used to keep completed jobs 
in memory. */
+    @Documentation.Section(Documentation.Sections.ALL_JOB_MANAGER)
+    public static final ConfigOption<Long> 
ARCHIVED_APPLICATION_STORE_CACHE_SIZE =
+            key("archived-application-store.cache-size")
+                    .longType()
+                    .defaultValue(50L * 1024L * 1024L)
+                    .withDescription(
+                            "The archived application store cache size in 
bytes which is used to keep completed applications in memory.");
+
+    /** The time in seconds after which a completed job expires and is purged 
from the job store. */
+    @Documentation.Section(Documentation.Sections.ALL_JOB_MANAGER)
+    public static final ConfigOption<Long> 
ARCHIVED_APPLICATION_STORE_EXPIRATION_TIME =
+            key("archived-application-store.expiration-time")
+                    .longType()
+                    .defaultValue(60L * 60L)

Review Comment:
   It's better to use `Duration` type for duration related configuration.
   see https://issues.apache.org/jira/browse/FLINK-35359



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

Reply via email to