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 2020/01/09 18:44:23 UTC

[GitHub] [flink] Myasuka opened a new pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Myasuka opened a new pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820
 
 
   ## What is the purpose of the change
   
   Due to the implemenation of write buffer manager of RocksDB and the issue cannot create stric capacity limit cache, we need to refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s).
   
   ## Brief change log
   
     - Introduce `RocksDBOperationUtils.java#calculateActualCacheSize` to calculate actual cache size.
     - Add test `testCreateSharedResourcesWithExpectedCapacity` to verify the capacity of cache created is what we want.
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - Added `testCreateSharedResourcesWithExpectedCapacity` to verify the capacity of cache created is what we want.
   
   ## 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)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365565010
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOperationUtils.java
 ##########
 @@ -213,4 +211,49 @@ public static void addColumnFamilyOptionsToCloseLater(
 			throw new IOException("Failed to acquire shared cache resource for RocksDB", e);
 		}
 	}
+
+	@VisibleForTesting
+	static RocksDBSharedResources allocateRocksDBSharedResources(long memorySize, double writeBufferRatio, double highPriorityPoolRatio) {
+		long calculatedCacheCapacity = calculateActualCacheCapacity(memorySize, writeBufferRatio);
+		final Cache cache = createCache(calculatedCacheCapacity, highPriorityPoolRatio);
+		long writeBufferManagerCapacity = calculateWriteBufferManagerCapacity(memorySize, writeBufferRatio);
+		final WriteBufferManager wbm = new WriteBufferManager(writeBufferManagerCapacity, cache);
+		return new RocksDBSharedResources(cache, wbm);
+	}
+
+	/**
+	 * Calculate the actual calculated memory size of cache, which would be shared among rocksDB instance(s).
 
 Review comment:
   Minor: the actual calculated memory size -> the actual memory size

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365564968
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBResourceContainerTest.java
 ##########
 @@ -145,12 +160,34 @@ public void testGetColumnFamilyOptionsWithSharedResources() throws Exception {
 		container.close();
 	}
 
+	@Test
+	public void testCreateSharedResourcesWithExpectedCapacity() throws Exception {
+		PowerMockito.spy(RocksDBOperationUtils.class);
+		final AtomicLong actualCacheCapacity = new AtomicLong(0L);
+		// the `createCache` wrapper is introduced due to PowerMockito cannot mock on native static method easily.
+		PowerMockito.when(RocksDBOperationUtils.createCache(anyLong(), anyDouble()))
+			.thenAnswer((Answer<LRUCache>) invocation -> {
+				Object[] arguments = invocation.getArguments();
+				actualCacheCapacity.set((long) arguments[0]);
+				return (LRUCache) invocation.callRealMethod();
+			});
+
+		long totalMemorySize = 2048L;
+		double writeBufferRatio = 0.5;
+		double highPriPoolRatio = 0.1;
+		createSharedResources(totalMemorySize, writeBufferRatio, highPriPoolRatio);
+		long expectedCacheCapacity = RocksDBOperationUtils.calculateActualCacheCapacity(totalMemorySize, writeBufferRatio);
+		assertThat(actualCacheCapacity.get(), is(expectedCacheCapacity));
 
 Review comment:
   I suggest to either remove this test, or also add a verification of write buffer manager size.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] Myasuka commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
Myasuka commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365275814
 
 

 ##########
 File path: docs/_includes/generated/rocks_db_configuration.html
 ##########
 @@ -30,7 +30,7 @@
             <td><h5>state.backend.rocksdb.memory.high-prio-pool-ratio</h5></td>
             <td style="word-wrap: break-word;">0.1</td>
             <td>Double</td>
-            <td>The fraction of cache memory that is reserved for high-priority data like index, filter, and compression dictionary blocks. This option only has an effect when 'state.backend.rocksdb.memory.managed' or 'state.backend.rocksdb.memory.fixed-per-slot' are configured.</td>
+            <td>The fraction of total shared memory that is reserved for high-priority data like index, filter, and compression dictionary blocks. This option only has an effect when 'state.backend.rocksdb.memory.managed' or 'state.backend.rocksdb.memory.fixed-per-slot' are configured.</td>
 
 Review comment:
   Thanks for pointing out, it should not be changed here since the ratio is really used by the cache.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143932940 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4268 TriggerType:PUSH TriggerID:584b04c3756dfd8723a103b8f1502b4bccf3a6a6
   Hash:584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144051158 TriggerType:PUSH TriggerID:584b04c3756dfd8723a103b8f1502b4bccf3a6a6
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   * 64d8a9a6821c95f904ca567afd7a056be7f04f3a Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143932940) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262) 
   * 584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144051158) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4268) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365226910
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOperationUtils.java
 ##########
 @@ -213,4 +211,40 @@ public static void addColumnFamilyOptionsToCloseLater(
 			throw new IOException("Failed to acquire shared cache resource for RocksDB", e);
 		}
 	}
+
+	@VisibleForTesting
+	static RocksDBSharedResources allocateRocksDBSharedResources(long size, double writeBufferRatio, double highPriorityPoolRatio) {
+		long calculatedCacheSize = calculateActualCacheSize(size, writeBufferRatio);
+		final Cache cache = createCache(calculatedCacheSize, highPriorityPoolRatio);
+		final WriteBufferManager wbm = new WriteBufferManager((long) (writeBufferRatio * size), cache);
 
 Review comment:
   The `WriteBufferManager` size should also be calculated with the deduced formula: `2 * total_memory_size * write_buffer_ratio / (3 + write_buffer_ratio)`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143932940 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   * 64d8a9a6821c95f904ca567afd7a056be7f04f3a Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143932940) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143932940 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:584b04c3756dfd8723a103b8f1502b4bccf3a6a6
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   * 64d8a9a6821c95f904ca567afd7a056be7f04f3a Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143932940) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262) 
   * 584b04c3756dfd8723a103b8f1502b4bccf3a6a6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572698084
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 (Thu Jan 09 18:46:37 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143932940 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   * 64d8a9a6821c95f904ca567afd7a056be7f04f3a Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143932940) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365127924
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java
 ##########
 @@ -121,7 +121,7 @@
 		.doubleType()
 		.defaultValue(0.1)
 		.withDescription(String.format(
-				"The fraction of cache memory that is reserved for high-priority data like index, filter, and " +
+				"The fraction of total shared memory that is reserved for high-priority data like index, filter, and " +
 
 Review comment:
   Ditto, please revert this change if no rational reason.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   * 64d8a9a6821c95f904ca567afd7a056be7f04f3a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365227931
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBResourceContainerTest.java
 ##########
 @@ -145,12 +160,34 @@ public void testGetColumnFamilyOptionsWithSharedResources() throws Exception {
 		container.close();
 	}
 
+	@Test
+	public void testCreateSharedResourcesWithExpectedCapacity() throws Exception {
 
 Review comment:
   The design of this test seems to be tightly coupled with the implementation, while it's true that we cannot get the actual cache size through RocksDB's JNI (not exposed so reflection couldn't get it, either)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143932940 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   * 64d8a9a6821c95f904ca567afd7a056be7f04f3a Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143932940) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365127569
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOperationUtils.java
 ##########
 @@ -213,4 +211,40 @@ public static void addColumnFamilyOptionsToCloseLater(
 			throw new IOException("Failed to acquire shared cache resource for RocksDB", e);
 		}
 	}
+
+	@VisibleForTesting
+	static RocksDBSharedResources allocateRocksDBSharedResources(long size, double writeBufferRatio, double highPriorityPoolRatio) {
+		long calculatedCacheSize = calculateActualCacheSize(size, writeBufferRatio);
+		final Cache cache = createCache(calculatedCacheSize, highPriorityPoolRatio);
+		final WriteBufferManager wbm = new WriteBufferManager((long) (writeBufferRatio * size), cache);
+		return new RocksDBSharedResources(cache, wbm);
+	}
+
+	/**
+	 * Calculate the actual calculated memory size of cache, which would be shared among rocksDB instance(s).
+	 * We introduce this method because:
+	 * a) We cannot create a strict capacity limit cache util FLINK-15532 resolved.
+	 * b) Regardless of the memory usage of blocks pinned by RocksDB iterators,
+	 * which is difficult to calculate and only happened when we iterator entries in RocksDBMapState, the overuse of memory is mainly occupied by at most half of the write buffer usage.
+	 * (see <a href="https://github.com/dataArtisans/frocksdb/blob/958f191d3f7276ae59b270f9db8390034d549ee0/include/rocksdb/write_buffer_manager.h#L51">the flush implementation of write buffer manager</a>).
+	 * Thus, we have four equations below:
+	 *   write_buffer_manager_memory = 1.5 * write_buffer_manager_capacity
+	 *   write_buffer_manager_memory = total_memory_size * write_buffer_ratio
+	 *   write_buffer_manager_memory + other_part = total_memory_size
+	 *   write_buffer_manager_capacity + other_part = cache_size
+	 * And we would deduce the formula: cache_size = 3 * total_memory_size / (3 + write_buffer_ratio)
+	 *
+	 * @param totalMemorySize  Total off-heap memory size reserved for RocksDB instance(s).
+	 * @param writeBufferRatio The ratio of memory size which could be reserved for write buffer manager to control the memory usage.
+	 * @return The actual calculated memory size.
+	 */
+	@VisibleForTesting
+	static long calculateActualCacheSize(long totalMemorySize, double writeBufferRatio) {
+		return (long) (3 * totalMemorySize / (3 + writeBufferRatio));
+	}
+
+	@VisibleForTesting
+	static Cache createCache(long cacheSize, double highPriorityPoolRatio) {
+		return new LRUCache(cacheSize, -1, false, highPriorityPoolRatio);
 
 Review comment:
   Suggest to add a TODO comment here, mentioning that we will change the `strictCapacityLimit` flag to `true` after [rocksdb#6247](https://github.com/facebook/rocksdb/issues/6247) is resolved.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365123307
 
 

 ##########
 File path: docs/_includes/generated/rocks_db_configuration.html
 ##########
 @@ -30,7 +30,7 @@
             <td><h5>state.backend.rocksdb.memory.high-prio-pool-ratio</h5></td>
             <td style="word-wrap: break-word;">0.1</td>
             <td>Double</td>
-            <td>The fraction of cache memory that is reserved for high-priority data like index, filter, and compression dictionary blocks. This option only has an effect when 'state.backend.rocksdb.memory.managed' or 'state.backend.rocksdb.memory.fixed-per-slot' are configured.</td>
+            <td>The fraction of total shared memory that is reserved for high-priority data like index, filter, and compression dictionary blocks. This option only has an effect when 'state.backend.rocksdb.memory.managed' or 'state.backend.rocksdb.memory.fixed-per-slot' are configured.</td>
 
 Review comment:
   Wondering why this description need to be changed, please clarify, thanks.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 commented on a change in pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#discussion_r365226910
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOperationUtils.java
 ##########
 @@ -213,4 +211,40 @@ public static void addColumnFamilyOptionsToCloseLater(
 			throw new IOException("Failed to acquire shared cache resource for RocksDB", e);
 		}
 	}
+
+	@VisibleForTesting
+	static RocksDBSharedResources allocateRocksDBSharedResources(long size, double writeBufferRatio, double highPriorityPoolRatio) {
+		long calculatedCacheSize = calculateActualCacheSize(size, writeBufferRatio);
+		final Cache cache = createCache(calculatedCacheSize, highPriorityPoolRatio);
+		final WriteBufferManager wbm = new WriteBufferManager((long) (writeBufferRatio * size), cache);
 
 Review comment:
   The `WriteBufferManager` size should also be calculated with the deduced formula: `2 * total_memory_size * write_buffer_ratio / 3`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143932940 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4268 TriggerType:PUSH TriggerID:584b04c3756dfd8723a103b8f1502b4bccf3a6a6
   Hash:584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/144051158 TriggerType:PUSH TriggerID:584b04c3756dfd8723a103b8f1502b4bccf3a6a6
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   * 64d8a9a6821c95f904ca567afd7a056be7f04f3a Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143932940) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262) 
   * 584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/144051158) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4268) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] carp84 merged pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
carp84 merged pull request #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10820: [FLINK-15512][statebackend] Refactor the mechanism to calculate the cache capacity shared among RocksDB instance(s)
URL: https://github.com/apache/flink/pull/10820#issuecomment-572715611
 
 
   <!--
   Meta data
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143787485 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241 TriggerType:PUSH TriggerID:bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143932940 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:64d8a9a6821c95f904ca567afd7a056be7f04f3a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262 TriggerType:PUSH TriggerID:64d8a9a6821c95f904ca567afd7a056be7f04f3a
   Hash:584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4268 TriggerType:PUSH TriggerID:584b04c3756dfd8723a103b8f1502b4bccf3a6a6
   Hash:584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144051158 TriggerType:PUSH TriggerID:584b04c3756dfd8723a103b8f1502b4bccf3a6a6
   -->
   ## CI report:
   
   * bfc6c4a4f7baa0bec85a9105b96014bb1fa5ce87 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143787485) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4241) 
   * 64d8a9a6821c95f904ca567afd7a056be7f04f3a Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143932940) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4262) 
   * 584b04c3756dfd8723a103b8f1502b4bccf3a6a6 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144051158) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4268) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services