You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/11/08 14:52:54 UTC

[GitHub] [flink] rkhachatryan opened a new pull request, #21264: [WIP][FLINK-29928][runtime, state] Share RocksDB memory across TM slots

rkhachatryan opened a new pull request, #21264:
URL: https://github.com/apache/flink/pull/21264

   ## What is the purpose of the change
   
   ```
   The memory shared is BlockCache and WriteBufferManager.
   
   The motivation is to allow more flexible memory distribution among tasks while still capping RocksDB memory usage.
   ```
   
   ## Verifying this change
   
   - end-to-end: `TaskManagerWideRocksDbMemorySharingITCase` using rocksdb metrics 
   **- unit tests: TBD**
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: yes
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? yes
     **- If yes, how is the feature documented? TBD**
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] AlanConfluent commented on a diff in pull request #21264: [WIP][FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
AlanConfluent commented on code in PR #21264:
URL: https://github.com/apache/flink/pull/21264#discussion_r1024790314


##########
flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.test.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.testutils.InMemoryReporter;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.math.BigInteger;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.DoubleSummaryStatistics;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.api.common.restartstrategy.RestartStrategies.noRestart;
+import static org.apache.flink.contrib.streaming.state.RocksDBMemoryControllerUtils.calculateActualCacheCapacity;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that memory sharing scope and {@link TaskManagerOptions#MANAGED_MEMORY_SHARED_FRACTION}
+ * work as expected, i.e. make RocksDB use the same BlockCache and WriteBufferManager objects. It
+ * does so using RocksDB metrics.
+ */
+public class TaskManagerWideRocksDbMemorySharingITCase {
+    private static final int PARALLELISM = 4;
+    private static final int NUMBER_OF_JOBS = 5;
+    private static final int NUMBER_OF_TASKS = NUMBER_OF_JOBS * PARALLELISM;
+
+    private static final int MANAGED_MEMORY_SIZE_BYTES = NUMBER_OF_TASKS * 25 * 1024 * 1024;
+    private static final double MANAGED_MEMORY_SHARED_FRACTION = .85d;
+    private static final double WRITE_BUFFER_RATIO = 0.5;
+    private static final double EXPECTED_BLOCK_CACHE_SIZE =
+            calculateActualCacheCapacity(
+                    (long) (MANAGED_MEMORY_SIZE_BYTES * MANAGED_MEMORY_SHARED_FRACTION),
+                    WRITE_BUFFER_RATIO);
+    // try to check that the memory usage is limited
+    // however, there is no hard limit actually
+    // because of https://issues.apache.org/jira/browse/FLINK-15532
+    private static final double EFFECTIVE_LIMIT = EXPECTED_BLOCK_CACHE_SIZE * 1.25;
+
+    private InMemoryReporter metricsReporter;
+    private MiniClusterWithClientResource cluster;
+
+    @Before
+    public void init() throws Exception {
+        metricsReporter = InMemoryReporter.create();
+        cluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(getConfiguration(metricsReporter))
+                                .setNumberTaskManagers(1)
+                                .setNumberSlotsPerTaskManager(NUMBER_OF_TASKS)
+                                .build());
+        cluster.before();
+    }
+
+    @After
+    public void destroy() {
+        cluster.after();
+        metricsReporter.close();
+    }
+
+    @Test
+    public void testBlockCache() throws Exception {
+        List<JobID> jobIDs = new ArrayList<>(NUMBER_OF_JOBS);
+        try {
+            // launch jobs
+            for (int i = 0; i < NUMBER_OF_JOBS; i++) {
+                jobIDs.add(cluster.getRestClusterClient().submitJob(dag()).get());
+            }
+
+            // wait for init
+            Deadline initDeadline = Deadline.fromNow(Duration.ofMinutes(1));
+            for (JobID jid : jobIDs) {
+                waitForAllTaskRunning(cluster.getMiniCluster(), jid, false);
+                waitForAllMetricsReported(jid, initDeadline);
+            }
+
+            // check declared capacity
+            collectGaugeValues(jobIDs, "rocksdb.block-cache-capacity")
+                    .forEach(
+                            size ->
+                                    assertEquals(
+                                            "Unexpected rocksdb block cache capacity",
+                                            EXPECTED_BLOCK_CACHE_SIZE,
+                                            size,
+                                            0));
+
+            // do some work and check the actual usage of memory
+            for (int i = 0; i < 10; i++) {
+                Thread.sleep(50L);
+                DoubleSummaryStatistics stats =
+                        collectGaugeValues(jobIDs, "rocksdb.block-cache-usage")
+                                .collect(Collectors.summarizingDouble((Double::doubleValue)));
+                assertEquals(
+                        String.format(
+                                "Block cache usage reported by different tasks varies too much: %s\n"

Review Comment:
   What's the idea behind verifying that the shared memory metric doesn't change much?  If you were accidentally allocating from exclusive memory, wouldn't you expect that this doesn't deviate much as well?  Or does this measure the cache used regardless of whether it's exclusive memory or not?



##########
flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.test.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.testutils.InMemoryReporter;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.math.BigInteger;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.DoubleSummaryStatistics;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.api.common.restartstrategy.RestartStrategies.noRestart;
+import static org.apache.flink.contrib.streaming.state.RocksDBMemoryControllerUtils.calculateActualCacheCapacity;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that memory sharing scope and {@link TaskManagerOptions#MANAGED_MEMORY_SHARED_FRACTION}
+ * work as expected, i.e. make RocksDB use the same BlockCache and WriteBufferManager objects. It
+ * does so using RocksDB metrics.
+ */
+public class TaskManagerWideRocksDbMemorySharingITCase {
+    private static final int PARALLELISM = 4;
+    private static final int NUMBER_OF_JOBS = 5;
+    private static final int NUMBER_OF_TASKS = NUMBER_OF_JOBS * PARALLELISM;
+
+    private static final int MANAGED_MEMORY_SIZE_BYTES = NUMBER_OF_TASKS * 25 * 1024 * 1024;
+    private static final double MANAGED_MEMORY_SHARED_FRACTION = .85d;
+    private static final double WRITE_BUFFER_RATIO = 0.5;
+    private static final double EXPECTED_BLOCK_CACHE_SIZE =
+            calculateActualCacheCapacity(
+                    (long) (MANAGED_MEMORY_SIZE_BYTES * MANAGED_MEMORY_SHARED_FRACTION),
+                    WRITE_BUFFER_RATIO);
+    // try to check that the memory usage is limited
+    // however, there is no hard limit actually
+    // because of https://issues.apache.org/jira/browse/FLINK-15532
+    private static final double EFFECTIVE_LIMIT = EXPECTED_BLOCK_CACHE_SIZE * 1.25;
+
+    private InMemoryReporter metricsReporter;
+    private MiniClusterWithClientResource cluster;
+
+    @Before
+    public void init() throws Exception {
+        metricsReporter = InMemoryReporter.create();
+        cluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(getConfiguration(metricsReporter))
+                                .setNumberTaskManagers(1)
+                                .setNumberSlotsPerTaskManager(NUMBER_OF_TASKS)
+                                .build());
+        cluster.before();
+    }
+
+    @After
+    public void destroy() {
+        cluster.after();
+        metricsReporter.close();
+    }
+
+    @Test
+    public void testBlockCache() throws Exception {
+        List<JobID> jobIDs = new ArrayList<>(NUMBER_OF_JOBS);
+        try {
+            // launch jobs
+            for (int i = 0; i < NUMBER_OF_JOBS; i++) {
+                jobIDs.add(cluster.getRestClusterClient().submitJob(dag()).get());
+            }
+
+            // wait for init
+            Deadline initDeadline = Deadline.fromNow(Duration.ofMinutes(1));
+            for (JobID jid : jobIDs) {
+                waitForAllTaskRunning(cluster.getMiniCluster(), jid, false);
+                waitForAllMetricsReported(jid, initDeadline);
+            }
+
+            // check declared capacity
+            collectGaugeValues(jobIDs, "rocksdb.block-cache-capacity")

Review Comment:
   Is this metric for the total shared memory used by rocks db and it doesn't matter if the memory is managed or not?



##########
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java:
##########
@@ -126,6 +128,7 @@ private SavepointEnvironment(
         this.taskStateManager = new SavepointTaskStateManager(prioritizedOperatorSubtaskState);
         this.ioManager = new IOManagerAsync(ConfigurationUtils.parseTempDirectories(configuration));
         this.memoryManager = MemoryManager.create(64 * 1024 * 1024, DEFAULT_PAGE_SIZE);
+        this.sharedMemoryManager = MemoryManager.create(0, DEFAULT_PAGE_SIZE);

Review Comment:
   Doesn't this create a memory manager with 0 pages? I assume this is not utilized in some of these environments. It seems like `TaskManagerServices` is where it's set to a real value.  Just curious, how is this environment used?



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] AlanConfluent commented on a diff in pull request #21264: [FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
AlanConfluent commented on code in PR #21264:
URL: https://github.com/apache/flink/pull/21264#discussion_r1033923674


##########
flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.test.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.testutils.InMemoryReporter;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.math.BigInteger;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.DoubleSummaryStatistics;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.api.common.restartstrategy.RestartStrategies.noRestart;
+import static org.apache.flink.contrib.streaming.state.RocksDBMemoryControllerUtils.calculateActualCacheCapacity;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that memory sharing scope and {@link TaskManagerOptions#MANAGED_MEMORY_SHARED_FRACTION}
+ * work as expected, i.e. make RocksDB use the same BlockCache and WriteBufferManager objects. It
+ * does so using RocksDB metrics.
+ */
+public class TaskManagerWideRocksDbMemorySharingITCase {
+    private static final int PARALLELISM = 4;
+    private static final int NUMBER_OF_JOBS = 5;
+    private static final int NUMBER_OF_TASKS = NUMBER_OF_JOBS * PARALLELISM;
+
+    private static final int MANAGED_MEMORY_SIZE_BYTES = NUMBER_OF_TASKS * 25 * 1024 * 1024;
+    private static final double MANAGED_MEMORY_SHARED_FRACTION = .85d;
+    private static final double WRITE_BUFFER_RATIO = 0.5;
+    private static final double EXPECTED_BLOCK_CACHE_SIZE =
+            calculateActualCacheCapacity(
+                    (long) (MANAGED_MEMORY_SIZE_BYTES * MANAGED_MEMORY_SHARED_FRACTION),
+                    WRITE_BUFFER_RATIO);
+    // try to check that the memory usage is limited
+    // however, there is no hard limit actually
+    // because of https://issues.apache.org/jira/browse/FLINK-15532
+    private static final double EFFECTIVE_LIMIT = EXPECTED_BLOCK_CACHE_SIZE * 1.25;
+
+    private InMemoryReporter metricsReporter;
+    private MiniClusterWithClientResource cluster;
+
+    @Before
+    public void init() throws Exception {
+        metricsReporter = InMemoryReporter.create();
+        cluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(getConfiguration(metricsReporter))
+                                .setNumberTaskManagers(1)
+                                .setNumberSlotsPerTaskManager(NUMBER_OF_TASKS)
+                                .build());
+        cluster.before();
+    }
+
+    @After
+    public void destroy() {
+        cluster.after();
+        metricsReporter.close();
+    }
+
+    @Test
+    public void testBlockCache() throws Exception {
+        List<JobID> jobIDs = new ArrayList<>(NUMBER_OF_JOBS);
+        try {
+            // launch jobs
+            for (int i = 0; i < NUMBER_OF_JOBS; i++) {
+                jobIDs.add(cluster.getRestClusterClient().submitJob(dag()).get());
+            }
+
+            // wait for init
+            Deadline initDeadline = Deadline.fromNow(Duration.ofMinutes(1));
+            for (JobID jid : jobIDs) {
+                waitForAllTaskRunning(cluster.getMiniCluster(), jid, false);
+                waitForAllMetricsReported(jid, initDeadline);
+            }
+
+            // check declared capacity
+            collectGaugeValues(jobIDs, "rocksdb.block-cache-capacity")
+                    .forEach(
+                            size ->
+                                    assertEquals(
+                                            "Unexpected rocksdb block cache capacity",
+                                            EXPECTED_BLOCK_CACHE_SIZE,
+                                            size,
+                                            0));
+
+            // do some work and check the actual usage of memory
+            for (int i = 0; i < 10; i++) {
+                Thread.sleep(50L);
+                DoubleSummaryStatistics stats =
+                        collectGaugeValues(jobIDs, "rocksdb.block-cache-usage")
+                                .collect(Collectors.summarizingDouble((Double::doubleValue)));
+                assertEquals(
+                        String.format(
+                                "Block cache usage reported by different tasks varies too much: %s\n"

Review Comment:
   That makes sense.  If it works in practice, then that works.  The alternative might be separate metrics for shared vs exclusive memory -- that might be useful if they are both in use.  Or is there some other way to access the internal state?  For these integration tests, it seems like only the public APIs are used, so maybe this isn't an option.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] rkhachatryan commented on a diff in pull request #21264: [FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #21264:
URL: https://github.com/apache/flink/pull/21264#discussion_r1034085757


##########
flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.test.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.testutils.InMemoryReporter;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.math.BigInteger;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.DoubleSummaryStatistics;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.api.common.restartstrategy.RestartStrategies.noRestart;
+import static org.apache.flink.contrib.streaming.state.RocksDBMemoryControllerUtils.calculateActualCacheCapacity;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that memory sharing scope and {@link TaskManagerOptions#MANAGED_MEMORY_SHARED_FRACTION}
+ * work as expected, i.e. make RocksDB use the same BlockCache and WriteBufferManager objects. It
+ * does so using RocksDB metrics.
+ */
+public class TaskManagerWideRocksDbMemorySharingITCase {
+    private static final int PARALLELISM = 4;
+    private static final int NUMBER_OF_JOBS = 5;
+    private static final int NUMBER_OF_TASKS = NUMBER_OF_JOBS * PARALLELISM;
+
+    private static final int MANAGED_MEMORY_SIZE_BYTES = NUMBER_OF_TASKS * 25 * 1024 * 1024;
+    private static final double MANAGED_MEMORY_SHARED_FRACTION = .85d;
+    private static final double WRITE_BUFFER_RATIO = 0.5;
+    private static final double EXPECTED_BLOCK_CACHE_SIZE =
+            calculateActualCacheCapacity(
+                    (long) (MANAGED_MEMORY_SIZE_BYTES * MANAGED_MEMORY_SHARED_FRACTION),
+                    WRITE_BUFFER_RATIO);
+    // try to check that the memory usage is limited
+    // however, there is no hard limit actually
+    // because of https://issues.apache.org/jira/browse/FLINK-15532
+    private static final double EFFECTIVE_LIMIT = EXPECTED_BLOCK_CACHE_SIZE * 1.25;
+
+    private InMemoryReporter metricsReporter;
+    private MiniClusterWithClientResource cluster;
+
+    @Before
+    public void init() throws Exception {
+        metricsReporter = InMemoryReporter.create();
+        cluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(getConfiguration(metricsReporter))
+                                .setNumberTaskManagers(1)
+                                .setNumberSlotsPerTaskManager(NUMBER_OF_TASKS)
+                                .build());
+        cluster.before();
+    }
+
+    @After
+    public void destroy() {
+        cluster.after();
+        metricsReporter.close();
+    }
+
+    @Test
+    public void testBlockCache() throws Exception {
+        List<JobID> jobIDs = new ArrayList<>(NUMBER_OF_JOBS);
+        try {
+            // launch jobs
+            for (int i = 0; i < NUMBER_OF_JOBS; i++) {
+                jobIDs.add(cluster.getRestClusterClient().submitJob(dag()).get());
+            }
+
+            // wait for init
+            Deadline initDeadline = Deadline.fromNow(Duration.ofMinutes(1));
+            for (JobID jid : jobIDs) {
+                waitForAllTaskRunning(cluster.getMiniCluster(), jid, false);
+                waitForAllMetricsReported(jid, initDeadline);
+            }
+
+            // check declared capacity
+            collectGaugeValues(jobIDs, "rocksdb.block-cache-capacity")
+                    .forEach(
+                            size ->
+                                    assertEquals(
+                                            "Unexpected rocksdb block cache capacity",
+                                            EXPECTED_BLOCK_CACHE_SIZE,
+                                            size,
+                                            0));
+
+            // do some work and check the actual usage of memory
+            for (int i = 0; i < 10; i++) {
+                Thread.sleep(50L);
+                DoubleSummaryStatistics stats =
+                        collectGaugeValues(jobIDs, "rocksdb.block-cache-usage")
+                                .collect(Collectors.summarizingDouble((Double::doubleValue)));
+                assertEquals(
+                        String.format(
+                                "Block cache usage reported by different tasks varies too much: %s\n"

Review Comment:
   Actually, shared memory metrics seem a good idea in general. 
   But adding them isn't super trivial (again configuration and passing the metrics should be thought through);
   so I'd do it separately, maybe after some feedback for this feature.
   And testing directly via task-level metrics seems a bit more robust.
   WDYT?
   
   There is no way to access RocksDB internal state at this level AFAIK.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] rkhachatryan commented on pull request #21264: [FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on PR #21264:
URL: https://github.com/apache/flink/pull/21264#issuecomment-1323953503

   Rebased and resolved the conflict.
   @AlanConfluent do you have any further feedback?


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] rkhachatryan commented on pull request #21264: [FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on PR #21264:
URL: https://github.com/apache/flink/pull/21264#issuecomment-1334144107

   Thanks for the review!
   Merged into master as 3b6d08e57f644cddcdac1fb5a110d44172652c3a.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] rkhachatryan merged pull request #21264: [FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
rkhachatryan merged PR #21264:
URL: https://github.com/apache/flink/pull/21264


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] rkhachatryan commented on a diff in pull request #21264: [WIP][FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #21264:
URL: https://github.com/apache/flink/pull/21264#discussion_r1024970422


##########
flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.test.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.testutils.InMemoryReporter;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.math.BigInteger;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.DoubleSummaryStatistics;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.api.common.restartstrategy.RestartStrategies.noRestart;
+import static org.apache.flink.contrib.streaming.state.RocksDBMemoryControllerUtils.calculateActualCacheCapacity;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that memory sharing scope and {@link TaskManagerOptions#MANAGED_MEMORY_SHARED_FRACTION}
+ * work as expected, i.e. make RocksDB use the same BlockCache and WriteBufferManager objects. It
+ * does so using RocksDB metrics.
+ */
+public class TaskManagerWideRocksDbMemorySharingITCase {
+    private static final int PARALLELISM = 4;
+    private static final int NUMBER_OF_JOBS = 5;
+    private static final int NUMBER_OF_TASKS = NUMBER_OF_JOBS * PARALLELISM;
+
+    private static final int MANAGED_MEMORY_SIZE_BYTES = NUMBER_OF_TASKS * 25 * 1024 * 1024;
+    private static final double MANAGED_MEMORY_SHARED_FRACTION = .85d;
+    private static final double WRITE_BUFFER_RATIO = 0.5;
+    private static final double EXPECTED_BLOCK_CACHE_SIZE =
+            calculateActualCacheCapacity(
+                    (long) (MANAGED_MEMORY_SIZE_BYTES * MANAGED_MEMORY_SHARED_FRACTION),
+                    WRITE_BUFFER_RATIO);
+    // try to check that the memory usage is limited
+    // however, there is no hard limit actually
+    // because of https://issues.apache.org/jira/browse/FLINK-15532
+    private static final double EFFECTIVE_LIMIT = EXPECTED_BLOCK_CACHE_SIZE * 1.25;
+
+    private InMemoryReporter metricsReporter;
+    private MiniClusterWithClientResource cluster;
+
+    @Before
+    public void init() throws Exception {
+        metricsReporter = InMemoryReporter.create();
+        cluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(getConfiguration(metricsReporter))
+                                .setNumberTaskManagers(1)
+                                .setNumberSlotsPerTaskManager(NUMBER_OF_TASKS)
+                                .build());
+        cluster.before();
+    }
+
+    @After
+    public void destroy() {
+        cluster.after();
+        metricsReporter.close();
+    }
+
+    @Test
+    public void testBlockCache() throws Exception {
+        List<JobID> jobIDs = new ArrayList<>(NUMBER_OF_JOBS);
+        try {
+            // launch jobs
+            for (int i = 0; i < NUMBER_OF_JOBS; i++) {
+                jobIDs.add(cluster.getRestClusterClient().submitJob(dag()).get());
+            }
+
+            // wait for init
+            Deadline initDeadline = Deadline.fromNow(Duration.ofMinutes(1));
+            for (JobID jid : jobIDs) {
+                waitForAllTaskRunning(cluster.getMiniCluster(), jid, false);
+                waitForAllMetricsReported(jid, initDeadline);
+            }
+
+            // check declared capacity
+            collectGaugeValues(jobIDs, "rocksdb.block-cache-capacity")
+                    .forEach(
+                            size ->
+                                    assertEquals(
+                                            "Unexpected rocksdb block cache capacity",
+                                            EXPECTED_BLOCK_CACHE_SIZE,
+                                            size,
+                                            0));
+
+            // do some work and check the actual usage of memory
+            for (int i = 0; i < 10; i++) {
+                Thread.sleep(50L);
+                DoubleSummaryStatistics stats =
+                        collectGaugeValues(jobIDs, "rocksdb.block-cache-usage")
+                                .collect(Collectors.summarizingDouble((Double::doubleValue)));
+                assertEquals(
+                        String.format(
+                                "Block cache usage reported by different tasks varies too much: %s\n"

Review Comment:
   The idea is that if all rocksdb instances are backed by the same block cache / write buffer manager,
   then the metric must show exactly the same usage (modulo reporting delay and measurement lags).
   
   If each rocksdb instance uses its own cache, then the usages will likely be different because of different state growth rates in each subtask.
   
   This is more of a heuristic, but I couldn't come up with a better way to check that there is only a single BlockCache/WBM behind the scenes.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] rkhachatryan commented on a diff in pull request #21264: [WIP][FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #21264:
URL: https://github.com/apache/flink/pull/21264#discussion_r1024964191


##########
flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.test.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.testutils.InMemoryReporter;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.math.BigInteger;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.DoubleSummaryStatistics;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.api.common.restartstrategy.RestartStrategies.noRestart;
+import static org.apache.flink.contrib.streaming.state.RocksDBMemoryControllerUtils.calculateActualCacheCapacity;
+import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that memory sharing scope and {@link TaskManagerOptions#MANAGED_MEMORY_SHARED_FRACTION}
+ * work as expected, i.e. make RocksDB use the same BlockCache and WriteBufferManager objects. It
+ * does so using RocksDB metrics.
+ */
+public class TaskManagerWideRocksDbMemorySharingITCase {
+    private static final int PARALLELISM = 4;
+    private static final int NUMBER_OF_JOBS = 5;
+    private static final int NUMBER_OF_TASKS = NUMBER_OF_JOBS * PARALLELISM;
+
+    private static final int MANAGED_MEMORY_SIZE_BYTES = NUMBER_OF_TASKS * 25 * 1024 * 1024;
+    private static final double MANAGED_MEMORY_SHARED_FRACTION = .85d;
+    private static final double WRITE_BUFFER_RATIO = 0.5;
+    private static final double EXPECTED_BLOCK_CACHE_SIZE =
+            calculateActualCacheCapacity(
+                    (long) (MANAGED_MEMORY_SIZE_BYTES * MANAGED_MEMORY_SHARED_FRACTION),
+                    WRITE_BUFFER_RATIO);
+    // try to check that the memory usage is limited
+    // however, there is no hard limit actually
+    // because of https://issues.apache.org/jira/browse/FLINK-15532
+    private static final double EFFECTIVE_LIMIT = EXPECTED_BLOCK_CACHE_SIZE * 1.25;
+
+    private InMemoryReporter metricsReporter;
+    private MiniClusterWithClientResource cluster;
+
+    @Before
+    public void init() throws Exception {
+        metricsReporter = InMemoryReporter.create();
+        cluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(getConfiguration(metricsReporter))
+                                .setNumberTaskManagers(1)
+                                .setNumberSlotsPerTaskManager(NUMBER_OF_TASKS)
+                                .build());
+        cluster.before();
+    }
+
+    @After
+    public void destroy() {
+        cluster.after();
+        metricsReporter.close();
+    }
+
+    @Test
+    public void testBlockCache() throws Exception {
+        List<JobID> jobIDs = new ArrayList<>(NUMBER_OF_JOBS);
+        try {
+            // launch jobs
+            for (int i = 0; i < NUMBER_OF_JOBS; i++) {
+                jobIDs.add(cluster.getRestClusterClient().submitJob(dag()).get());
+            }
+
+            // wait for init
+            Deadline initDeadline = Deadline.fromNow(Duration.ofMinutes(1));
+            for (JobID jid : jobIDs) {
+                waitForAllTaskRunning(cluster.getMiniCluster(), jid, false);
+                waitForAllMetricsReported(jid, initDeadline);
+            }
+
+            // check declared capacity
+            collectGaugeValues(jobIDs, "rocksdb.block-cache-capacity")

Review Comment:
   That's right, it measures the total **capacity** (among all backends on TM), regardless of how the memory was allocated.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] flinkbot commented on pull request #21264: [WIP][FLINK-29928][runtime, state] Share RocksDB memory across TM slots

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #21264:
URL: https://github.com/apache/flink/pull/21264#issuecomment-1307355412

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "755d3770d267c71283e41597926fc3a8b6ace118",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "755d3770d267c71283e41597926fc3a8b6ace118",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 755d3770d267c71283e41597926fc3a8b6ace118 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

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