rkhachatryan commented on code in PR #19598:
URL: https://github.com/apache/flink/pull/19598#discussion_r867395555


##########
docs/content.zh/docs/ops/metrics.md:
##########
@@ -1277,7 +1277,7 @@ Note that the metrics are only available via reporters.
   </thead>
   <tbody>
     <tr>
-      <th rowspan="20"><strong>Job (only available on 
TaskManager)</strong></th>
+      <th rowspan="8"><strong>Job (only available on TaskManager)</strong></th>

Review Comment:
   This change should probably be replicated in the Chinese version.



##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java:
##########
@@ -672,12 +685,10 @@ public void updateChangelogSnapshotState(
                 subtaskName,
                 upTo,
                 materializedSnapshot);
+        List<KeyedStateHandle> materializedResult = 
getMaterializedResult(materializedSnapshot);

Review Comment:
   nit: this variable can be inlined back



##########
docs/content.zh/docs/ops/metrics.md:
##########
@@ -1317,6 +1317,42 @@ Note that the metrics are only available via reporters.
       <td>Current size of upload queue. Queue items can be packed together and 
form a single upload.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <th rowspan="7"><strong>Task/Operator</strong></th>
+      <td>startedMaterialization</td>
+      <td>The number of started materialization.</td>
+      <td>Counter</td>
+    </tr>
+    <tr>
+      <td>completedMaterialization</td>
+      <td>The number of successfully completed materialization.</td>
+      <td>Counter</td>
+    </tr>
+    <tr>
+      <td>failedMaterialization</td>
+      <td>The number of failed materialization.</td>
+      <td>Counter</td>
+    </tr>
+    <tr>
+      <td>lastFullSizeOfMaterialization</td>
+      <td>The full size of the last materialization part (in bytes).</td>

Review Comment:
   This is actually last reported checkpoint, not last materialization, right?
   How about
   "The full size of the materialization part of the last reported checkpoint 
(in bytes)."
   ?
   
   ditto: other added metrics.



##########
docs/content.zh/docs/ops/metrics.md:
##########
@@ -1317,6 +1317,42 @@ Note that the metrics are only available via reporters.
       <td>Current size of upload queue. Queue items can be packed together and 
form a single upload.</td>
       <td>Gauge</td>
     </tr>
+    <tr>
+      <th rowspan="7"><strong>Task/Operator</strong></th>
+      <td>startedMaterialization</td>
+      <td>The number of started materialization.</td>

Review Comment:
   The number of started materialization[s].
   ?
   
   ditto: other added metrics



##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java:
##########
@@ -132,8 +136,14 @@ public synchronized void close() {
     public void triggerMaterialization() {
         mailboxExecutor.execute(
                 () -> {
-                    Optional<MaterializationRunnable> 
materializationRunnableOptional =
-                            keyedStateBackend.initMaterialization();
+                    metrics.reportStartedMaterialization();
+                    Optional<MaterializationRunnable> 
materializationRunnableOptional;
+                    try {
+                        materializationRunnableOptional = 
keyedStateBackend.initMaterialization();
+                    } catch (Exception ex) {
+                        metrics.reportFailedMaterialization();
+                        throw ex;
+                    }
 
                     if (materializationRunnableOptional.isPresent()) {

Review Comment:
   In the opposite case (optional not present), shouldn't completed 
materialization be reported? Or maybe empty materialization?
   My reasoning is that with no state updates, the number of started 
materializations might be much higher than the sum of completed and failed.



##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java:
##########
@@ -387,10 +393,17 @@ public RunnableFuture<SnapshotResult<KeyedStateHandle>> 
snapshot(
                                         buildSnapshotResult(
                                                 checkpointId,
                                                 delta,
-                                                
changelogStateBackendStateCopy)));
+                                                
changelogStateBackendStateCopy))
+                        .whenComplete(
+                                (snapshotResult, throwable) ->
+                                        
metrics.reportSnapshotResult(snapshotResult))

Review Comment:
   This can be executed by a writer thread (upon finishing the upload);
   or by the task thread, if the uploads are already finished.
   
   So this call should **probably** be syncrhonized.
   
   The common approach in Flink is to not synchronize metric writes with reads, 
and allow reporting stale metrics. But here, the update might happen from a 
different thread, which might lead to inconsistencies (e.g. mixing sizes from 
different checkpoitns).
   
   Adding synchornization shouldn't incur performance penalty, as this code 
exeutes infrequently.
   
   So how about marking `reportSnapshotResults` method `synchronized`?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestUtils.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+import org.apache.flink.util.function.FunctionWithException;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.concurrent.RunnableFuture;
+import java.util.stream.Stream;
+
+/** This class contains test utils of {@link StateBackend} */
+public class StateBackendTestUtils {

Review Comment:
   Shouldn't this class be a singleton, and have a private constructor?



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