flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From se...@apache.org
Subject [5/5] flink git commit: [hotfix] [rocksdb] Update predefined RocksDB option profiles.
Date Tue, 16 Feb 2016 18:28:15 GMT
[hotfix] [rocksdb] Update predefined RocksDB option profiles.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/75dcfc02
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/75dcfc02
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/75dcfc02

Branch: refs/heads/master
Commit: 75dcfc02736a9b4d4a407a1348a761ff132ec23f
Parents: 489e653
Author: Stephan Ewen <sewen@apache.org>
Authored: Mon Feb 15 17:40:30 2016 +0100
Committer: Stephan Ewen <sewen@apache.org>
Committed: Tue Feb 16 17:42:18 2016 +0100

----------------------------------------------------------------------
 .../streaming/state/PredefinedOptions.java      | 105 +++++++++++++++++--
 1 file changed, 97 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/75dcfc02/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
index 383f043..c19b54f 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.contrib.streaming.state;
 
+import org.rocksdb.BlockBasedTableConfig;
 import org.rocksdb.CompactionStyle;
 import org.rocksdb.Options;
 
@@ -32,13 +33,19 @@ import org.rocksdb.Options;
 public enum PredefinedOptions {
 
 	/**
-	 * Default options for all settings.
+	 * Default options for all settings, except that writes are not forced to the
+	 * disk.
+	 * 
+	 * <p>Note: Because Flink does not rely on RocksDB data on disk for recovery,
+	 * there is no need to sync data to stable storage.
 	 */
 	DEFAULT {
 		
 		@Override
 		public Options createOptions() {
-			return new Options();
+			return new Options()
+					.setUseFsync(false)
+					.setDisableDataSync(true);
 		}
 	},
 
@@ -47,36 +54,118 @@ public enum PredefinedOptions {
 	 * 
 	 * <p>This constant configures RocksDB with some options that lead empirically
 	 * to better performance when the machines executing the system use
-	 * regular spinning hard disks. The following options are set:
+	 * regular spinning hard disks.
+	 * 
+	 * <p>The following options are set:
 	 * <ul>
-	 *     <li>Optimized level-style compactions</li>
+	 *     <li>setCompactionStyle(CompactionStyle.LEVEL)</li>
+	 *     <li>setLevelCompactionDynamicLevelBytes(true)</li>
+	 *     <li>setIncreaseParallelism(4)</li>
+	 *     <li>setUseFsync(false)</li>
+	 *     <li>setDisableDataSync(true)</li>
+	 *     <li>setMaxOpenFiles(-1)</li>
 	 * </ul>
+	 * 
+	 * <p>Note: Because Flink does not rely on RocksDB data on disk for recovery,
+	 * there is no need to sync data to stable storage.
 	 */
 	SPINNING_DISK_OPTIMIZED {
 
 		@Override
 		public Options createOptions() {
+
 			return new Options()
 					.setCompactionStyle(CompactionStyle.LEVEL)
-					.optimizeLevelStyleCompaction();
+					.setLevelCompactionDynamicLevelBytes(true)
+					.setIncreaseParallelism(4)
+					.setUseFsync(false)
+					.setDisableDataSync(true)
+					.setMaxOpenFiles(-1);
 		}
 	},
 
 	/**
+	 * Pre-defined options for better performance on regular spinning hard disks,
+	 * at the cost of a higher memory consumption.
+	 *
+	 * <p><b>NOTE: These settings will cause RocksDB to consume a lot of memory
for
+	 * block caching and compactions. If you experience out-of-memory problems related to,
+	 * RocksDB, consider switching back to {@link #SPINNING_DISK_OPTIMIZED}.</b></p>
+	 *
+	 * <p>The following options are set:
+	 * <ul>
+	 *     <li>setLevelCompactionDynamicLevelBytes(true)</li>
+	 *     <li>setTargetFileSizeBase(256 MBytes)</li>
+	 *     <li>setMaxBytesForLevelBase(1 GByte)</li>
+	 *     <li>setWriteBufferSize(64 MBytes)</li>
+	 *     <li>setIncreaseParallelism(4)</li>
+	 *     <li>setMinWriteBufferNumberToMerge(3)</li>
+	 *     <li>setMaxWriteBufferNumber(4)</li>
+	 *     <li>setUseFsync(false)</li>
+	 *     <li>setMaxOpenFiles(-1)</li>
+	 *     <li>BlockBasedTableConfig.setBlockCacheSize(256 MBytes)</li>
+	 *     <li>BlockBasedTableConfigsetBlockSize(128 KBytes)</li>
+	 * </ul>
+	 * 
+	 * <p>Note: Because Flink does not rely on RocksDB data on disk for recovery,
+	 * there is no need to sync data to stable storage.
+	 */
+	SPINNING_DISK_OPTIMIZED_HIGH_MEM {
+
+		@Override
+		public Options createOptions() {
+
+			final long blockCacheSize = 256 * 1024 * 1024;
+			final long blockSize = 128 * 1024;
+			final long targetFileSize = 256 * 1024 * 1024;
+			final long writeBufferSize = 64 * 1024 * 1024;
+
+			return new Options()
+					.setCompactionStyle(CompactionStyle.LEVEL)
+					.setLevelCompactionDynamicLevelBytes(true)
+					.setTargetFileSizeBase(targetFileSize)
+					.setMaxBytesForLevelBase(4 * targetFileSize)
+					.setWriteBufferSize(writeBufferSize)
+					.setIncreaseParallelism(4)
+					.setMinWriteBufferNumberToMerge(3)
+					.setMaxWriteBufferNumber(4)
+					.setUseFsync(false)
+					.setDisableDataSync(true)
+					.setMaxOpenFiles(-1)
+					.setTableFormatConfig(
+							new BlockBasedTableConfig()
+									.setBlockCacheSize(blockCacheSize)
+									.setBlockSize(blockSize)
+					);
+		}
+	},
+	
+	/**
 	 * Pre-defined options for Flash SSDs.
 	 *
 	 * <p>This constant configures RocksDB with some options that lead empirically
 	 * to better performance when the machines executing the system use SSDs.
-	 * The following options are set:
+	 * 
+	 * <p>The following options are set:
 	 * <ul>
-	 *     <li>none</li>
+	 *     <li>setIncreaseParallelism(4)</li>
+	 *     <li>setUseFsync(false)</li>
+	 *     <li>setDisableDataSync(true)</li>
+	 *     <li>setMaxOpenFiles(-1)</li>
 	 * </ul>
+	 * 
+	 * <p>Note: Because Flink does not rely on RocksDB data on disk for recovery,
+	 * there is no need to sync data to stable storage.
 	 */
 	FLASH_SSD_OPTIMIZED {
 
 		@Override
 		public Options createOptions() {
-			return new Options();
+			return new Options()
+					.setIncreaseParallelism(4)
+					.setUseFsync(false)
+					.setDisableDataSync(true)
+					.setMaxOpenFiles(-1);
 		}
 	};
 	


Mime
View raw message