You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ku...@apache.org on 2018/10/25 19:09:02 UTC

[drill] branch master updated: DRILL-6715: Update descriptions for System Options table

This is an automated email from the ASF dual-hosted git repository.

kunal pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new 61e8b46  DRILL-6715: Update descriptions for System Options table
61e8b46 is described below

commit 61e8b464063299dc1f67445157a46c4939b0cace
Author: Kunal Khatua <kk...@users.noreply.github.com>
AuthorDate: Thu Oct 25 12:08:57 2018 -0700

    DRILL-6715: Update descriptions for System Options table
    
    With introduction of DRILL-5735 , the descriptions for about half the system options still remain missing. This commit collects descriptions review by @bbevens
    
    1. Update options for HashAgg/Join	(@Ben-Zvi )
    2. Update options for Parquet Reader/Writer	(@sachouche )
    3. Update options for Planners	(@HanumathRao , @vdiravka , @KazydubB )
    4. Update options for BatchSizing	(@bitblender )
    5. Update options for Planner Optimizations	(@arina-ielchiieva )
    6. Update options for Security & Kafka	(Krystal Nguyen)
    7. Update options for Misc entries	(@arina-ielchiieva , @vvysotskyi )
    
    In additional, there is a patch for `org.apache.drill.exec.compile.ClassTransformer.scalar_replacement` , which appears to have replaced `exec.compile.scalar_replacement`. References to the latter have been removed to avoid confusion.
    Additional changes include moving the `ClassTransformer` validator to `ExecConstants.java`
    Adding support for internal options' descriptions
    Removed mention of {{Will be removed in 1.15.0}}. (Refer DRILL-6527)
---
 .../java/org/apache/drill/exec/ExecConstants.java  | 207 +++++++++++++--------
 .../drill/exec/compile/ClassTransformer.java       |   9 +-
 .../exec/planner/physical/PlannerSettings.java     |  31 +--
 .../exec/server/options/SystemOptionManager.java   |   3 +-
 .../apache/drill/exec/server/rest/WebServer.java   |   3 +
 .../java-exec/src/main/resources/drill-module.conf |   1 -
 .../java/org/apache/drill/test/QueryTestUtil.java  |   6 +-
 7 files changed, 158 insertions(+), 102 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 03707b2..21e16eb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -91,7 +91,8 @@ public final class ExecConstants {
 
   // Based on available memory, adjust output batch size for buffered operators by this factor.
   public static final String OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR = "drill.exec.memory.operator.output_batch_size_avail_mem_factor";
-  public static final DoubleValidator OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR_VALIDATOR = new RangeDoubleValidator(OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR, 0.01, 1.0, null);
+  public static final DoubleValidator OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR_VALIDATOR = new RangeDoubleValidator(OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR, 0.01, 1.0,
+      new OptionDescription("Based on the available system memory, adjusts the output batch size for buffered operators by the factor set."));
 
   // External Sort Boot configuration
 
@@ -115,25 +116,35 @@ public final class ExecConstants {
 
   // Hash Join Options
   public static final String HASHJOIN_HASHTABLE_CALC_TYPE_KEY = "exec.hashjoin.hash_table_calc_type";
-  public static final StringValidator HASHJOIN_HASHTABLE_CALC_TYPE = new StringValidator(HASHJOIN_HASHTABLE_CALC_TYPE_KEY, null);
+  public static final EnumeratedStringValidator HASHJOIN_HASHTABLE_CALC_TYPE = new EnumeratedStringValidator(HASHJOIN_HASHTABLE_CALC_TYPE_KEY,
+      new OptionDescription("Sets the Hash Join Memory Calculator type. Default is LEAN. This option also accepts CONSERVATIVE as a value."),
+      "LEAN", "CONSERVATIVE");
   public static final String HASHJOIN_SAFETY_FACTOR_KEY = "exec.hashjoin.safety_factor";
-  public static final DoubleValidator HASHJOIN_SAFETY_FACTOR = new RangeDoubleValidator(HASHJOIN_SAFETY_FACTOR_KEY, 1.0, Double.MAX_VALUE, null);
+  public static final DoubleValidator HASHJOIN_SAFETY_FACTOR = new RangeDoubleValidator(HASHJOIN_SAFETY_FACTOR_KEY, 1.0, Double.MAX_VALUE,
+      new OptionDescription("Sets the Hash Join Memory Calculation Safety; multiplies the internal size estimate. Default is 1.0"));
   public static final String HASHJOIN_HASH_DOUBLE_FACTOR_KEY = "exec.hashjoin.hash_double_factor";
-  public static final DoubleValidator HASHJOIN_HASH_DOUBLE_FACTOR = new RangeDoubleValidator(HASHJOIN_HASH_DOUBLE_FACTOR_KEY, 1.0, Double.MAX_VALUE, null);
+  public static final DoubleValidator HASHJOIN_HASH_DOUBLE_FACTOR = new RangeDoubleValidator(HASHJOIN_HASH_DOUBLE_FACTOR_KEY, 1.0, Double.MAX_VALUE,
+      new OptionDescription("Sets the Hash Join Memory Calculation; doubling factor for the Hash-Table. Default is 2.0"));
   public static final String HASHJOIN_FRAGMENTATION_FACTOR_KEY = "exec.hashjoin.fragmentation_factor";
-  public static final DoubleValidator HASHJOIN_FRAGMENTATION_FACTOR = new RangeDoubleValidator(HASHJOIN_FRAGMENTATION_FACTOR_KEY, 1.0, Double.MAX_VALUE, null);
+  public static final DoubleValidator HASHJOIN_FRAGMENTATION_FACTOR = new RangeDoubleValidator(HASHJOIN_FRAGMENTATION_FACTOR_KEY, 1.0, Double.MAX_VALUE,
+      new OptionDescription("Sets the Hash Join Memory Calculations; multiplies the internal estimates to account for fragmentation. Default is 1.33"));
   public static final String HASHJOIN_NUM_ROWS_IN_BATCH_KEY = "exec.hashjoin.num_rows_in_batch";
-  public static final LongValidator HASHJOIN_NUM_ROWS_IN_BATCH_VALIDATOR = new RangeLongValidator(HASHJOIN_NUM_ROWS_IN_BATCH_KEY, 1, 65536, null);
+  public static final LongValidator HASHJOIN_NUM_ROWS_IN_BATCH_VALIDATOR = new RangeLongValidator(HASHJOIN_NUM_ROWS_IN_BATCH_KEY, 1, 65536,
+      new OptionDescription("Sets the number of rows in the internal batches for Hash Join operations. Default is 1024"));
   public static final String HASHJOIN_MAX_BATCHES_IN_MEMORY_KEY = "exec.hashjoin.max_batches_in_memory";
-  public static final LongValidator HASHJOIN_MAX_BATCHES_IN_MEMORY_VALIDATOR = new RangeLongValidator(HASHJOIN_MAX_BATCHES_IN_MEMORY_KEY, 0, 65536, null);
+  public static final LongValidator HASHJOIN_MAX_BATCHES_IN_MEMORY_VALIDATOR = new RangeLongValidator(HASHJOIN_MAX_BATCHES_IN_MEMORY_KEY, 0, 65536,
+      new OptionDescription("Sets the maximum number of batches allowed in memory before spilling is enforced for Hash Join operations; used for testing purposes."));
   public static final String HASHJOIN_NUM_PARTITIONS_KEY = "exec.hashjoin.num_partitions";
-  public static final LongValidator HASHJOIN_NUM_PARTITIONS_VALIDATOR = new RangeLongValidator(HASHJOIN_NUM_PARTITIONS_KEY, 1, 128, null); // 1 means - no spilling
+  public static final LongValidator HASHJOIN_NUM_PARTITIONS_VALIDATOR = new RangeLongValidator(HASHJOIN_NUM_PARTITIONS_KEY, 1, 128,
+      new OptionDescription("Sets the initial number of internal partitions for Hash Join operations. Default is 32. May reduce when memory is too small. Disables spilling if set to 1.")); // 1 means - no spilling
   public static final String HASHJOIN_MAX_MEMORY_KEY = "drill.exec.hashjoin.mem_limit";
-  public static final LongValidator HASHJOIN_MAX_MEMORY_VALIDATOR = new RangeLongValidator(HASHJOIN_MAX_MEMORY_KEY, 0L, Long.MAX_VALUE, null);
+  public static final LongValidator HASHJOIN_MAX_MEMORY_VALIDATOR = new RangeLongValidator(HASHJOIN_MAX_MEMORY_KEY, 0L, Long.MAX_VALUE,
+      new OptionDescription("Enforces the maximum memory limit for the Hash Join operator (if non-zero); used for testing purposes. Default is 0 (disabled)."));
   public static final String HASHJOIN_SPILL_DIRS = "drill.exec.hashjoin.spill.directories";
   public static final String HASHJOIN_SPILL_FILESYSTEM = "drill.exec.hashjoin.spill.fs";
   public static final String HASHJOIN_FALLBACK_ENABLED_KEY = "drill.exec.hashjoin.fallback.enabled";
-  public static final BooleanValidator HASHJOIN_FALLBACK_ENABLED_VALIDATOR = new BooleanValidator(HASHJOIN_FALLBACK_ENABLED_KEY, null);
+  public static final BooleanValidator HASHJOIN_FALLBACK_ENABLED_VALIDATOR = new BooleanValidator(HASHJOIN_FALLBACK_ENABLED_KEY,
+      new OptionDescription("Hash Joins ignore memory limits when this option is enabled (true). When disabled (false), Hash Joins fail when memory is set too low."));
   public static final String HASHJOIN_ENABLE_RUNTIME_FILTER_KEY = "exec.hashjoin.enable.runtime_filter";
   public static final BooleanValidator HASHJOIN_ENABLE_RUNTIME_FILTER = new BooleanValidator(HASHJOIN_ENABLE_RUNTIME_FILTER_KEY, null);
   public static final String HASHJOIN_BLOOM_FILTER_MAX_SIZE_KEY = "exec.hashjoin.bloom_filter.max.size";
@@ -145,23 +156,29 @@ public final class ExecConstants {
 
   // Hash Aggregate Options
   public static final String HASHAGG_NUM_PARTITIONS_KEY = "exec.hashagg.num_partitions";
-  public static final LongValidator HASHAGG_NUM_PARTITIONS_VALIDATOR = new RangeLongValidator(HASHAGG_NUM_PARTITIONS_KEY, 1, 128, null); // 1 means - no spilling
+  public static final LongValidator HASHAGG_NUM_PARTITIONS_VALIDATOR = new RangeLongValidator(HASHAGG_NUM_PARTITIONS_KEY, 1, 128,
+      new OptionDescription("Sets the initial number of internal partitions for Hash Aggregates. Default is 32. May reduce when memory is too small. Disables spilling if set to 1.")); // 1 means - no spilling
   public static final String HASHAGG_MAX_MEMORY_KEY = "exec.hashagg.mem_limit";
-  public static final LongValidator HASHAGG_MAX_MEMORY_VALIDATOR = new RangeLongValidator(HASHAGG_MAX_MEMORY_KEY, 0, Integer.MAX_VALUE, null);
+  public static final LongValidator HASHAGG_MAX_MEMORY_VALIDATOR = new RangeLongValidator(HASHAGG_MAX_MEMORY_KEY, 0, Integer.MAX_VALUE,
+      new OptionDescription("Enforces the value set as the maximum memory for the Hash Aggregates. Default is 0 (disabled)."));
   // min batches is used for tuning (each partition needs so many batches when planning the number of partitions,
   // or reserve this number when calculating whether the remaining available memory is too small and requires a spill.)
   // Low value may OOM (e.g., when incoming rows become wider), higher values use fewer partitions but are safer
   public static final String HASHAGG_MIN_BATCHES_PER_PARTITION_KEY = "exec.hashagg.min_batches_per_partition";
-  public static final LongValidator HASHAGG_MIN_BATCHES_PER_PARTITION_VALIDATOR = new RangeLongValidator(HASHAGG_MIN_BATCHES_PER_PARTITION_KEY, 1, 5, null);
+  public static final LongValidator HASHAGG_MIN_BATCHES_PER_PARTITION_VALIDATOR = new RangeLongValidator(HASHAGG_MIN_BATCHES_PER_PARTITION_KEY, 1, 5,
+      new OptionDescription("Sets the safety assumption for the minimum number of batches needed for each partition when performing hash aggregation. Default is 2. "
+          + "Low value may OOM if incoming rows become very wide. See Spill-to-disk for Hash Aggregate operator for more information."));
   // Can be turned off mainly for testing. Memory prediction is used to decide on when to spill to disk; with this option off,
   // spill would be triggered only by another mechanism -- "catch OOMs and then spill".
   public static final String HASHAGG_USE_MEMORY_PREDICTION_KEY = "exec.hashagg.use_memory_prediction";
-  public static final BooleanValidator HASHAGG_USE_MEMORY_PREDICTION_VALIDATOR = new BooleanValidator(HASHAGG_USE_MEMORY_PREDICTION_KEY, null);
+  public static final BooleanValidator HASHAGG_USE_MEMORY_PREDICTION_VALIDATOR = new BooleanValidator(HASHAGG_USE_MEMORY_PREDICTION_KEY,
+      new OptionDescription("Enables Hash Aggregates to use memory predictions to proactively spill early. Default is true."));
 
   public static final String HASHAGG_SPILL_DIRS = "drill.exec.hashagg.spill.directories";
   public static final String HASHAGG_SPILL_FILESYSTEM = "drill.exec.hashagg.spill.fs";
   public static final String HASHAGG_FALLBACK_ENABLED_KEY = "drill.exec.hashagg.fallback.enabled";
-  public static final BooleanValidator HASHAGG_FALLBACK_ENABLED_VALIDATOR = new BooleanValidator(HASHAGG_FALLBACK_ENABLED_KEY, null);
+  public static final BooleanValidator HASHAGG_FALLBACK_ENABLED_VALIDATOR = new BooleanValidator(HASHAGG_FALLBACK_ENABLED_KEY,
+      new OptionDescription("Hash Aggregates ignore memory limits when enabled (true). When disabled (false), Hash Aggregates fail when memory is set too low."));
 
   public static final String SSL_PROVIDER = "drill.exec.ssl.provider"; // valid values are "JDK", "OPENSSL" // default JDK
   public static final String SSL_PROTOCOL = "drill.exec.ssl.protocol"; // valid values are SSL, SSLV2, SSLV3, TLS, TLSV1, TLSv1.1, TLSv1.2(default)
@@ -269,13 +286,14 @@ public final class ExecConstants {
   public static final OptionValidator OUTPUT_FORMAT_VALIDATOR = new StringValidator(OUTPUT_FORMAT_OPTION,
       new OptionDescription("Output format for data written to tables with the CREATE TABLE AS (CTAS) command. Allowed values are parquet, json, psv, csv, or tsv."));
   public static final String PARQUET_WRITER_USE_SINGLE_FS_BLOCK = "store.parquet.writer.use_single_fs_block";
-  public static final OptionValidator PARQUET_WRITER_USE_SINGLE_FS_BLOCK_VALIDATOR = new BooleanValidator(
-    PARQUET_WRITER_USE_SINGLE_FS_BLOCK, null);
+  public static final OptionValidator PARQUET_WRITER_USE_SINGLE_FS_BLOCK_VALIDATOR = new BooleanValidator(PARQUET_WRITER_USE_SINGLE_FS_BLOCK,
+      new OptionDescription("Instructs the Parquet writer to create files with the configured block size (instead of the default filesystem block size)."));
   public static final String PARQUET_BLOCK_SIZE = "store.parquet.block-size";
   public static final OptionValidator PARQUET_BLOCK_SIZE_VALIDATOR = new PositiveLongValidator(PARQUET_BLOCK_SIZE, Integer.MAX_VALUE,
-      new OptionDescription("Sets the size of a Parquet row group to the number of bytes less than or equal to the block size of MFS, HDFS, or the file system."));
+      new OptionDescription("Sets the row group size (in bytes) for output Parquet files to a number that is less than or equal to the block size in MFS, HDFS, or the file system."));
   public static final String PARQUET_PAGE_SIZE = "store.parquet.page-size";
-  public static final OptionValidator PARQUET_PAGE_SIZE_VALIDATOR = new PositiveLongValidator(PARQUET_PAGE_SIZE, Integer.MAX_VALUE, null);
+  public static final OptionValidator PARQUET_PAGE_SIZE_VALIDATOR = new PositiveLongValidator(PARQUET_PAGE_SIZE, Integer.MAX_VALUE,
+      new OptionDescription("Sets the page size for output Parquet files."));
   public static final String PARQUET_DICT_PAGE_SIZE = "store.parquet.dictionary.page-size";
   public static final OptionValidator PARQUET_DICT_PAGE_SIZE_VALIDATOR = new PositiveLongValidator(PARQUET_DICT_PAGE_SIZE, Integer.MAX_VALUE,
       new OptionDescription("For internal use. Do not change."));
@@ -287,20 +305,27 @@ public final class ExecConstants {
       PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING,
       new OptionDescription("For internal use. Do not change."));
 
-  public static final String PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS
-      = "store.parquet.writer.use_primitive_types_for_decimals";
-  public static final OptionValidator PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS_VALIDATOR = new BooleanValidator(
-    PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS, null);
+  public static final String PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS = "store.parquet.writer.use_primitive_types_for_decimals";
+  public static final OptionValidator PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS_VALIDATOR = new BooleanValidator(PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS,
+      new OptionDescription("Instructs the Parquet writer to convert decimal to primitive types whenever possible."));
 
-  public static final String PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS
-      = "store.parquet.writer.logical_type_for_decimals";
-  public static final OptionValidator PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS_VALIDATOR
-      = new EnumeratedStringValidator(PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, null, "fixed_len_byte_array", "binary");
+  public static final String PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS = "store.parquet.writer.logical_type_for_decimals";
+  public static final OptionValidator PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS_VALIDATOR = new EnumeratedStringValidator(PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS,
+      new OptionDescription("Parquet writer logical type for decimal; supported types \'fixed_len_byte_array\' and \'binary\'"),
+      "fixed_len_byte_array", "binary");
 
+  // TODO - The below two options don't seem to be used in the Drill code base
+  @Deprecated // TODO: DRILL-6527
   public static final String PARQUET_VECTOR_FILL_THRESHOLD = "store.parquet.vector_fill_threshold";
-  public static final OptionValidator PARQUET_VECTOR_FILL_THRESHOLD_VALIDATOR = new PositiveLongValidator(PARQUET_VECTOR_FILL_THRESHOLD, 99L, null);
+  @Deprecated // TODO: DRILL-6527
+  public static final OptionValidator PARQUET_VECTOR_FILL_THRESHOLD_VALIDATOR = new PositiveLongValidator(PARQUET_VECTOR_FILL_THRESHOLD, 99L,
+      new OptionDescription("Deprecated."));
+  @Deprecated // TODO: DRILL-6527
   public static final String PARQUET_VECTOR_FILL_CHECK_THRESHOLD = "store.parquet.vector_fill_check_threshold";
-  public static final OptionValidator PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR = new PositiveLongValidator(PARQUET_VECTOR_FILL_CHECK_THRESHOLD, 100L, null);
+  @Deprecated // TODO: DRILL-6527
+  public static final OptionValidator PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR = new PositiveLongValidator(PARQUET_VECTOR_FILL_CHECK_THRESHOLD, 100L,
+      new OptionDescription("Deprecated."));
+
   public static final String PARQUET_NEW_RECORD_READER = "store.parquet.use_new_reader";
   public static final OptionValidator PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR = new BooleanValidator(PARQUET_NEW_RECORD_READER,
       new OptionDescription("Not supported in this release."));
@@ -314,10 +339,12 @@ public final class ExecConstants {
 
   // Number of pages the Async Parquet page reader will read before blocking
   public static final String PARQUET_PAGEREADER_QUEUE_SIZE = "store.parquet.reader.pagereader.queuesize";
-  public static final OptionValidator PARQUET_PAGEREADER_QUEUE_SIZE_VALIDATOR = new  PositiveLongValidator(PARQUET_PAGEREADER_QUEUE_SIZE, Integer.MAX_VALUE, null);
+  public static final OptionValidator PARQUET_PAGEREADER_QUEUE_SIZE_VALIDATOR = new  PositiveLongValidator(PARQUET_PAGEREADER_QUEUE_SIZE, Integer.MAX_VALUE,
+      new OptionDescription("Sets the number of pages that the Parquet reader prefetches per column."));
 
   public static final String PARQUET_PAGEREADER_ENFORCETOTALSIZE = "store.parquet.reader.pagereader.enforceTotalSize";
-  public static final OptionValidator PARQUET_PAGEREADER_ENFORCETOTALSIZE_VALIDATOR = new BooleanValidator(PARQUET_PAGEREADER_ENFORCETOTALSIZE, null);
+  public static final OptionValidator PARQUET_PAGEREADER_ENFORCETOTALSIZE_VALIDATOR = new BooleanValidator(PARQUET_PAGEREADER_ENFORCETOTALSIZE,
+      new OptionDescription("Instructs the Parquet reader to read no more than the advertised page size."));
 
   public static final String PARQUET_COLUMNREADER_ASYNC = "store.parquet.reader.columnreader.async";
   public static final OptionValidator PARQUET_COLUMNREADER_ASYNC_VALIDATOR = new BooleanValidator(PARQUET_COLUMNREADER_ASYNC,
@@ -338,30 +365,41 @@ public final class ExecConstants {
   public static final OptionValidator PARQUET_PAGEREADER_USE_FADVISE_VALIDATOR = new  BooleanValidator(PARQUET_PAGEREADER_USE_FADVISE,
       new OptionDescription("If the file system supports it, the Parquet file reader issues an fadvise call to enable file server side sequential reading and caching. Since many HDFS implementations do not support this and because this may have no effect in conditions of high concurrency, the option is set to false. Useful for benchmarks and for performance critical queries."));
 
-  public static final OptionValidator COMPILE_SCALAR_REPLACEMENT = new BooleanValidator("exec.compile.scalar_replacement", null);
+  // scalar replacement strategy
+  public final static String SCALAR_REPLACEMENT_OPTION = "org.apache.drill.exec.compile.ClassTransformer.scalar_replacement";
+  public final static EnumeratedStringValidator SCALAR_REPLACEMENT_VALIDATOR = new EnumeratedStringValidator( SCALAR_REPLACEMENT_OPTION,
+      new OptionDescription("Enables Drill to attempt scalar replacement. If an error occurs during the attempt, Drill falls back to the previous behavior. Default is 'try'. Accepted values are 'try', 'on', and 'off'. (Drill 0.8+)"),
+      "try", "on", "off");
 
   // Controls whether to enable bulk parquet reader processing
   public static final String PARQUET_FLAT_READER_BULK = "store.parquet.flat.reader.bulk";
-  public static final OptionValidator PARQUET_FLAT_READER_BULK_VALIDATOR = new BooleanValidator(PARQUET_FLAT_READER_BULK, null);
+  public static final OptionValidator PARQUET_FLAT_READER_BULK_VALIDATOR = new BooleanValidator(PARQUET_FLAT_READER_BULK,
+      new OptionDescription("Parquet Reader which uses bulk processing (default)."));
 
   // Controls the flat parquet reader batching constraints (number of record and memory limit)
   public static final String PARQUET_FLAT_BATCH_NUM_RECORDS = "store.parquet.flat.batch.num_records";
-  public static final OptionValidator PARQUET_FLAT_BATCH_NUM_RECORDS_VALIDATOR = new RangeLongValidator(PARQUET_FLAT_BATCH_NUM_RECORDS, 1, ValueVector.MAX_ROW_COUNT, null);
+  public static final OptionValidator PARQUET_FLAT_BATCH_NUM_RECORDS_VALIDATOR = new RangeLongValidator(PARQUET_FLAT_BATCH_NUM_RECORDS, 1, ValueVector.MAX_ROW_COUNT,
+      new OptionDescription("Parquet Reader maximum number of records per batch."));
   public static final String PARQUET_FLAT_BATCH_MEMORY_SIZE = "store.parquet.flat.batch.memory_size";
   // This configuration is used to overwrite the common memory batch sizing configuration property
-  public static final OptionValidator PARQUET_FLAT_BATCH_MEMORY_SIZE_VALIDATOR = new RangeLongValidator(PARQUET_FLAT_BATCH_MEMORY_SIZE, 0, Integer.MAX_VALUE, null);
+  public static final OptionValidator PARQUET_FLAT_BATCH_MEMORY_SIZE_VALIDATOR = new RangeLongValidator(PARQUET_FLAT_BATCH_MEMORY_SIZE, 0, Integer.MAX_VALUE,
+      new OptionDescription("Parquet Reader maximum memory size per batch."));
 
   public static final String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";
   public static final BooleanValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE,
       new OptionDescription("Drill reads all data from the JSON files as VARCHAR. Prevents schema change errors."));
   public static final BooleanValidator JSON_EXTENDED_TYPES = new BooleanValidator("store.json.extended_types",
       new OptionDescription("Turns on special JSON structures that Drill serializes for storing more type information than the four basic JSON types."));
-  public static final BooleanValidator JSON_WRITER_UGLIFY = new BooleanValidator("store.json.writer.uglify", null);
-  public static final BooleanValidator JSON_WRITER_SKIPNULLFIELDS = new BooleanValidator("store.json.writer.skip_null_fields", null);
+  public static final BooleanValidator JSON_WRITER_UGLIFY = new BooleanValidator("store.json.writer.uglify",
+      new OptionDescription("Enables Drill to return compact JSON output files; Drill does not separate records. Default is false. (Drill 1.4+)"));
+  public static final BooleanValidator JSON_WRITER_SKIPNULLFIELDS = new BooleanValidator("store.json.writer.skip_null_fields",
+      new OptionDescription("Enables Drill to skip extraneous NULL fields in JSON output files when executing the CTAS statement. Default is true. (Drill 1.6+)"));
   public static final String JSON_READER_SKIP_INVALID_RECORDS_FLAG = "store.json.reader.skip_invalid_records";
-  public static final BooleanValidator JSON_SKIP_MALFORMED_RECORDS_VALIDATOR = new BooleanValidator(JSON_READER_SKIP_INVALID_RECORDS_FLAG, null);
+  public static final BooleanValidator JSON_SKIP_MALFORMED_RECORDS_VALIDATOR = new BooleanValidator(JSON_READER_SKIP_INVALID_RECORDS_FLAG,
+      new OptionDescription("Allows queries to progress when the JSON record reader skips bad records in JSON files. Default is false. (Drill 1.9+)"));
   public static final String JSON_READER_PRINT_INVALID_RECORDS_LINE_NOS_FLAG = "store.json.reader.print_skipped_invalid_record_number";
-  public static final BooleanValidator JSON_READER_PRINT_INVALID_RECORDS_LINE_NOS_FLAG_VALIDATOR = new BooleanValidator(JSON_READER_PRINT_INVALID_RECORDS_LINE_NOS_FLAG, null);
+  public static final BooleanValidator JSON_READER_PRINT_INVALID_RECORDS_LINE_NOS_FLAG_VALIDATOR = new BooleanValidator(JSON_READER_PRINT_INVALID_RECORDS_LINE_NOS_FLAG,
+      new OptionDescription("Enables Drill to log the bad records that the JSON record reader skips when reading JSON files. Default is false. (Drill 1.9+)"));
   public static final DoubleValidator TEXT_ESTIMATED_ROW_SIZE = new RangeDoubleValidator("store.text.estimated_row_size_bytes", 1, Long.MAX_VALUE,
       new OptionDescription("Estimate of the row size in a delimited text file, such as csv. The closer to actual, the better the query plan. Used for all csv files in the system/session where the value is set. Impacts the decision to plan a broadcast join or not."));
 
@@ -370,12 +408,14 @@ public final class ExecConstants {
    * Json writer option for writing `NaN` and `Infinity` tokens as numbers (not enclosed with double quotes)
    */
   public static final String JSON_WRITER_NAN_INF_NUMBERS = "store.json.writer.allow_nan_inf";
-  public static final BooleanValidator JSON_WRITER_NAN_INF_NUMBERS_VALIDATOR = new BooleanValidator(JSON_WRITER_NAN_INF_NUMBERS, null);
+  public static final BooleanValidator JSON_WRITER_NAN_INF_NUMBERS_VALIDATOR = new BooleanValidator(JSON_WRITER_NAN_INF_NUMBERS,
+      new OptionDescription("Enables the JSON writer in Drill to write `NaN` and `Infinity` tokens as numbers (not enclosed with double quotes) to a JSON file. Default is true. (Drill 1.13+)"));
   /**
    * Json reader option that enables parser to read `NaN` and `Infinity` tokens as numbers
    */
   public static final String JSON_READER_NAN_INF_NUMBERS = "store.json.reader.allow_nan_inf";
-  public static final BooleanValidator JSON_READER_NAN_INF_NUMBERS_VALIDATOR = new BooleanValidator(JSON_READER_NAN_INF_NUMBERS, null);
+  public static final BooleanValidator JSON_READER_NAN_INF_NUMBERS_VALIDATOR = new BooleanValidator(JSON_READER_NAN_INF_NUMBERS,
+      new OptionDescription("Enables the JSON record reader in Drill to read `NaN` and `Infinity` tokens in JSON data as numbers. Default is true. (Drill 1.13+)"));
   /**
    * The column label (for directory levels) in results when querying files in a directory
    * E.g.  labels: dir0   dir1<pre>
@@ -428,28 +468,31 @@ public final class ExecConstants {
   public static final OptionValidator KAFKA_READER_READ_NUMBERS_AS_DOUBLE_VALIDATOR = new BooleanValidator(
       KAFKA_READER_READ_NUMBERS_AS_DOUBLE, new OptionDescription("Similar to store.json.read_numbers_as_double."));
   public static final String KAFKA_RECORD_READER = "store.kafka.record.reader";
-  public static final OptionValidator KAFKA_RECORD_READER_VALIDATOR = new StringValidator(KAFKA_RECORD_READER, null);
+  public static final OptionValidator KAFKA_RECORD_READER_VALIDATOR = new StringValidator(KAFKA_RECORD_READER,
+      new OptionDescription("The Kafka record reader configured to read incoming messages from Kafka."));
   public static final String KAFKA_POLL_TIMEOUT = "store.kafka.poll.timeout";
-  public static final PositiveLongValidator KAFKA_POLL_TIMEOUT_VALIDATOR = new PositiveLongValidator(KAFKA_POLL_TIMEOUT,
-      Long.MAX_VALUE, null);
+  public static final PositiveLongValidator KAFKA_POLL_TIMEOUT_VALIDATOR = new PositiveLongValidator(KAFKA_POLL_TIMEOUT, Long.MAX_VALUE,
+      new OptionDescription("Amount of time in milliseconds allotted to the Kafka client to fetch messages from the Kafka cluster; default value is 200."));
 
   // TODO: We need to add a feature that enables storage plugins to add their own options. Currently we have to declare
   // in core which is not right. Move this option and above two mongo plugin related options once we have the feature.
-  @Deprecated // TODO: DRILL-6527. It should be removed starting from next Drill 1.15.0 release
+  @Deprecated // TODO: DRILL-6527
   public static final String HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS = "store.hive.optimize_scan_with_native_readers";
-  @Deprecated // TODO: DRILL-6527. It should be removed starting from next Drill 1.15.0 release
-  public static final OptionValidator HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS_VALIDATOR =
-      new BooleanValidator(HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS, null);
+  @Deprecated // TODO: DRILL-6527
+  public static final OptionValidator HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS_VALIDATOR = new BooleanValidator(HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS,
+      new OptionDescription("Deprecated as of Drill 1.14. Use the store.hive.parquet.optimize_scan_with_native_reader option instead. Enables Drill to use the Drill native reader (instead of the Hive Serde interface) to optimize reads of Parquet-backed tables from Hive. Default is false."));
   public static final String HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER = "store.hive.parquet.optimize_scan_with_native_reader";
   public static final OptionValidator HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER_VALIDATOR =
       new BooleanValidator(HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER,
-          new OptionDescription("Optimize reads of Parquet-backed external tables from Hive by using Drill native readers instead of the Hive Serde interface. (Drill 1.2 and later)"));
+          new OptionDescription("Optimize reads of Parquet-backed external tables from Hive by using Drill native readers instead of the Hive Serde interface. (Drill 1.2+)"));
   public static final String HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER = "store.hive.maprdb_json.optimize_scan_with_native_reader";
   public static final OptionValidator HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER_VALIDATOR =
-      new BooleanValidator(HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER, null);
+      new BooleanValidator(HIVE_OPTIMIZE_MAPRDB_JSON_SCAN_WITH_NATIVE_READER,
+          new OptionDescription("Enables Drill to use the Drill native reader (instead of the Hive Serde interface) to optimize reads of MapR Database JSON tables from Hive. Default is false. (Drill 1.14+)"));
 
   public static final String HIVE_CONF_PROPERTIES = "store.hive.conf.properties";
-  public static final OptionValidator HIVE_CONF_PROPERTIES_VALIDATOR = new StringValidator(HIVE_CONF_PROPERTIES, null);
+  public static final OptionValidator HIVE_CONF_PROPERTIES_VALIDATOR = new StringValidator(HIVE_CONF_PROPERTIES,
+      new OptionDescription("Enables the user to specify Hive properties at the session level. Do not set the property values in quotes. Separate the property name and value by =. Separate each property with a new line (\\n). Example: set `store.hive.conf.properties` = 'hive.mapred.supports.subdirectories=true\\nmapred.input.dir.recursive=true'. (Drill 1.14+)"));
 
   public static final String SLICE_TARGET = "planner.slice_target";
   public static final long SLICE_TARGET_DEFAULT = 100000L;
@@ -498,10 +541,12 @@ public final class ExecConstants {
       new OptionDescription("Factor by which a node with endpoint affinity will be favored while creating assignment"));
 
   public static final String EARLY_LIMIT0_OPT_KEY = "planner.enable_limit0_optimization";
-  public static final BooleanValidator EARLY_LIMIT0_OPT = new BooleanValidator(EARLY_LIMIT0_OPT_KEY, null);
+  public static final BooleanValidator EARLY_LIMIT0_OPT = new BooleanValidator(EARLY_LIMIT0_OPT_KEY,
+      new OptionDescription("Sets the type of identifier quotes for the SQL parser. Default is backticks ('`'). The SQL parser accepts double quotes ('\"') and square brackets ('['). (Drill 1.11+)"));
 
   public static final String LATE_LIMIT0_OPT_KEY = "planner.enable_limit0_on_scan";
-  public static final BooleanValidator LATE_LIMIT0_OPT = new BooleanValidator(LATE_LIMIT0_OPT_KEY, null);
+  public static final BooleanValidator LATE_LIMIT0_OPT = new BooleanValidator(LATE_LIMIT0_OPT_KEY,
+      new OptionDescription("Enables Drill to determine data types as Drill scans data. This optimization is used when the query planner cannot infer types of columns during validation (prior to scanning). Drill exits and terminates the query immediately after resolving the types. When this optimization is applied, the query plan contains a LIMIT (0) above every SCAN, with an optional PROJECT in between. Default is true. (Drill 1.14+)"));
 
   public static final String ENABLE_MEMORY_ESTIMATION_KEY = "planner.memory.enable_memory_estimation";
   public static final OptionValidator ENABLE_MEMORY_ESTIMATION = new BooleanValidator(ENABLE_MEMORY_ESTIMATION_KEY,
@@ -563,8 +608,8 @@ public final class ExecConstants {
    */
   public static final String NON_BLOCKING_OPERATORS_MEMORY_KEY = "planner.memory.non_blocking_operators_memory";
   public static final OptionValidator NON_BLOCKING_OPERATORS_MEMORY = new PowerOfTwoLongValidator(
-    NON_BLOCKING_OPERATORS_MEMORY_KEY, 1 << 11,
-    new OptionDescription("Extra query memory per node for non-blocking operators. This option is currently used only for memory estimation. Range: 0-2048 MB"));
+      NON_BLOCKING_OPERATORS_MEMORY_KEY, 1 << 11,
+      new OptionDescription("Extra query memory per node for non-blocking operators. This option is currently used only for memory estimation. Range: 0-2048 MB"));
 
   public static final String HASH_JOIN_TABLE_FACTOR_KEY = "planner.memory.hash_join_table_factor";
   public static final OptionValidator HASH_JOIN_TABLE_FACTOR = new DoubleValidator(HASH_JOIN_TABLE_FACTOR_KEY,
@@ -582,7 +627,6 @@ public final class ExecConstants {
   public static final String ORDERED_MUX_EXCHANGE = "planner.enable_ordered_mux_exchange";
 
   // Resource management boot-time options.
-
   public static final String MAX_MEMORY_PER_NODE = "drill.exec.rm.memory_per_node";
   public static final String MAX_CPUS_PER_NODE = "drill.exec.rm.cpus_per_node";
 
@@ -625,7 +669,7 @@ public final class ExecConstants {
 
   public static final String ENABLE_WINDOW_FUNCTIONS = "window.enable";
   public static final OptionValidator ENABLE_WINDOW_FUNCTIONS_VALIDATOR = new BooleanValidator(ENABLE_WINDOW_FUNCTIONS,
-      new OptionDescription("Enable or disable window functions in Drill 1.1 and later."));
+      new OptionDescription("Enable or disable window functions in Drill 1.1+"));
 
   public static final String DRILLBIT_CONTROL_INJECTIONS = "drill.exec.testing.controls";
   public static final OptionValidator DRILLBIT_CONTROLS_VALIDATOR = new ExecutionControls.ControlsOptionValidator(DRILLBIT_CONTROL_INJECTIONS, 1, null);
@@ -654,14 +698,15 @@ public final class ExecConstants {
    * such as changing system options.
    */
   public static final String ADMIN_USERS_KEY = "security.admin.users";
-  public static final AdminUsersValidator ADMIN_USERS_VALIDATOR = new AdminUsersValidator(ADMIN_USERS_KEY, null);
+  public static final AdminUsersValidator ADMIN_USERS_VALIDATOR = new AdminUsersValidator(ADMIN_USERS_KEY,
+      new OptionDescription("A comma-separated list of user names with administrator privileges."));
 
   /**
    * Option whose value is a comma separated list of admin usergroups.
    */
   public static final String ADMIN_USER_GROUPS_KEY = "security.admin.user_groups";
   public static final AdminUserGroupsValidator ADMIN_USER_GROUPS_VALIDATOR =
-          new AdminUserGroupsValidator(ADMIN_USER_GROUPS_KEY, null);
+      new AdminUserGroupsValidator(ADMIN_USER_GROUPS_KEY, new OptionDescription("A comma-separated list of user groups with administrator privileges."));
   /**
    * Option whose value is a string representing list of inbound impersonation policies.
    *
@@ -683,28 +728,29 @@ public final class ExecConstants {
    * Web settings
    */
   public static final String WEB_LOGS_MAX_LINES = "web.logs.max_lines";
-  public static final OptionValidator WEB_LOGS_MAX_LINES_VALIDATOR = new PositiveLongValidator(WEB_LOGS_MAX_LINES, Integer.MAX_VALUE, null);
+  public static final OptionValidator WEB_LOGS_MAX_LINES_VALIDATOR = new PositiveLongValidator(WEB_LOGS_MAX_LINES, Integer.MAX_VALUE,
+      new OptionDescription("Provides the maximum number of log file lines that display on the Logs tab in the Drill Web UI. (Drill 1.7+)"));
 
   public static final String WEB_DISPLAY_FORMAT_TIMESTAMP = "web.display_format.timestamp";
   public static final OptionValidator WEB_DISPLAY_FORMAT_TIMESTAMP_VALIDATOR = new DateTimeFormatValidator(WEB_DISPLAY_FORMAT_TIMESTAMP,
       new OptionDescription("Display format template for timestamp. "
-                          + "It will be passed to java.time.format.DateTimeFormatter. "
-                          + "See https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html for the details about acceptable patterns. "
-                          + "If empty then the default formatting will be used. (Drill 1.15 and later)"));
+          + "It will be passed to java.time.format.DateTimeFormatter. "
+          + "See https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html for the details about acceptable patterns. "
+          + "If empty then the default formatting will be used. (Drill 1.15+)"));
 
   public static final String WEB_DISPLAY_FORMAT_DATE = "web.display_format.date";
   public static final OptionValidator WEB_DISPLAY_FORMAT_DATE_VALIDATOR = new DateTimeFormatValidator(WEB_DISPLAY_FORMAT_DATE,
       new OptionDescription("Display format template for date. "
-                          + "It will be passed to java.time.format.DateTimeFormatter. "
-                          + "See https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html for the details about acceptable patterns. "
-                          + "If empty then the default formatting will be used. (Drill 1.15 and later)"));
+          + "It will be passed to java.time.format.DateTimeFormatter. "
+          + "See https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html for the details about acceptable patterns. "
+          + "If empty then the default formatting will be used. (Drill 1.15+)"));
 
   public static final String WEB_DISPLAY_FORMAT_TIME = "web.display_format.time";
   public static final OptionValidator WEB_DISPLAY_FORMAT_TIME_VALIDATOR = new DateTimeFormatValidator(WEB_DISPLAY_FORMAT_TIME,
       new OptionDescription("Display format template for time. "
-                          + "It will be passed to java.time.format.DateTimeFormatter. "
-                          + "See https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html for the details about acceptable patterns. "
-                          + "If empty then the default formatting will be used. (Drill 1.15 and later)"));
+          + "It will be passed to java.time.format.DateTimeFormatter. "
+          + "See https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html for the details about acceptable patterns. "
+          + "If empty then the default formatting will be used. (Drill 1.15+)"));
 
   public static final String CODE_GEN_EXP_IN_METHOD_SIZE = "exec.java.compiler.exp_in_method_size";
   public static final LongValidator CODE_GEN_EXP_IN_METHOD_SIZE_VALIDATOR = new LongValidator(CODE_GEN_EXP_IN_METHOD_SIZE,
@@ -715,7 +761,8 @@ public final class ExecConstants {
       new PositiveLongValidator(CREATE_PREPARE_STATEMENT_TIMEOUT_MILLIS, Integer.MAX_VALUE, null);
 
   public static final String DYNAMIC_UDF_SUPPORT_ENABLED = "exec.udf.enable_dynamic_support";
-  public static final BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED, null);
+  public static final BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED,
+      new OptionDescription("Enables users to dynamically upload UDFs. Users must upload their UDF (source and binary) JAR files to a staging directory in the distributed file system before issuing the CREATE FUNCTION USING JAR command to register a UDF. Default is true. (Drill 1.9+)"));
 
   /**
    * Option to save query profiles. If false, no query profile will be saved
@@ -734,13 +781,15 @@ public final class ExecConstants {
   public static final BooleanValidator QUERY_PROFILE_DEBUG_VALIDATOR = new BooleanValidator(QUERY_PROFILE_DEBUG_OPTION, null);
 
   public static final String USE_DYNAMIC_UDFS_KEY = "exec.udf.use_dynamic";
-  public static final BooleanValidator USE_DYNAMIC_UDFS = new BooleanValidator(USE_DYNAMIC_UDFS_KEY, null);
+  public static final BooleanValidator USE_DYNAMIC_UDFS = new BooleanValidator(USE_DYNAMIC_UDFS_KEY,
+      new OptionDescription("Enables Drill to build an operator table for built-in static functions and then reuse the table across queries. Drill uses the operator table for all queries that do not need dynamic UDF support. The operator table does not include dynamic UDFs. Default is true. (Drill 1.10+)"));
 
   public static final String QUERY_TRANSIENT_STATE_UPDATE_KEY = "exec.query.progress.update";
   public static final BooleanValidator QUERY_TRANSIENT_STATE_UPDATE = new BooleanValidator(QUERY_TRANSIENT_STATE_UPDATE_KEY, null);
 
   public static final String PERSISTENT_TABLE_UMASK = "exec.persistent_table.umask";
-  public static final StringValidator PERSISTENT_TABLE_UMASK_VALIDATOR = new StringValidator(PERSISTENT_TABLE_UMASK, null);
+  public static final StringValidator PERSISTENT_TABLE_UMASK_VALIDATOR = new StringValidator(PERSISTENT_TABLE_UMASK,
+      new OptionDescription("Enables users to modify permissions on directories and files that result from running the CTAS command. The default is 002, which sets the default directory permissions to 775 and default file permissions to 664. (Drill 1.11+)"));
 
   /**
    * Enables batch iterator (operator) validation. Validation is normally enabled
@@ -776,7 +825,7 @@ public final class ExecConstants {
 
   public static String bootDefaultFor(String name) {
     return OPTION_DEFAULTS_ROOT + name;
-}
+  }
   /**
    * Boot-time config option provided to modify duration of the grace period.
    * Grace period is the amount of time where the drillbit accepts work after
@@ -794,16 +843,20 @@ public final class ExecConstants {
 
   /** Enables batch size statistics logging */
   public static final String STATS_LOGGING_BATCH_SIZE_OPTION = "drill.exec.stats.logging.batch_size";
-  public static final BooleanValidator STATS_LOGGING_BATCH_SIZE_VALIDATOR = new BooleanValidator(STATS_LOGGING_BATCH_SIZE_OPTION, null);
+  public static final BooleanValidator STATS_LOGGING_BATCH_SIZE_VALIDATOR = new BooleanValidator(STATS_LOGGING_BATCH_SIZE_OPTION,
+      new OptionDescription("Enables batch size statistics logging."));
 
   /** Enables fine-grained batch size statistics logging */
   public static final String STATS_LOGGING_FG_BATCH_SIZE_OPTION = "drill.exec.stats.logging.fine_grained.batch_size";
-  public static final BooleanValidator STATS_LOGGING_BATCH_FG_SIZE_VALIDATOR = new BooleanValidator(STATS_LOGGING_FG_BATCH_SIZE_OPTION, null);
+  public static final BooleanValidator STATS_LOGGING_BATCH_FG_SIZE_VALIDATOR = new BooleanValidator(STATS_LOGGING_FG_BATCH_SIZE_OPTION,
+      new OptionDescription("Enables fine-grained batch size statistics logging."));
 
   /** Controls the list of operators for which batch sizing stats should be enabled */
   public static final String STATS_LOGGING_BATCH_OPERATOR_OPTION = "drill.exec.stats.logging.enabled_operators";
-  public static final StringValidator STATS_LOGGING_BATCH_OPERATOR_VALIDATOR = new StringValidator(STATS_LOGGING_BATCH_OPERATOR_OPTION, null);
+  public static final StringValidator STATS_LOGGING_BATCH_OPERATOR_VALIDATOR = new StringValidator(STATS_LOGGING_BATCH_OPERATOR_OPTION,
+      new OptionDescription("Controls the list of operators for which batch sizing statistics should be enabled."));
 
   public static final String LIST_FILES_RECURSIVELY = "storage.list_files_recursively";
-  public static final BooleanValidator LIST_FILES_RECURSIVELY_VALIDATOR = new BooleanValidator(LIST_FILES_RECURSIVELY, null);
+  public static final BooleanValidator LIST_FILES_RECURSIVELY_VALIDATOR = new BooleanValidator(LIST_FILES_RECURSIVELY,
+      new OptionDescription("Enables recursive files listing when querying the `INFORMATION_SCHEMA.FILES` table or executing the SHOW FILES command. Default is false. (Drill 1.15+"));
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
index 05548da..8248d73 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
@@ -26,11 +26,11 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.common.util.DrillStringUtils;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.compile.MergeAdapter.MergedClassResult;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.server.options.OptionSet;
-import org.apache.drill.exec.server.options.TypeValidators.EnumeratedStringValidator;
 import org.codehaus.commons.compiler.CompileException;
 import org.objectweb.asm.ClassReader;
 import org.objectweb.asm.tree.ClassNode;
@@ -58,11 +58,6 @@ public class ClassTransformer {
   private final DrillConfig config;
   private final OptionSet optionManager;
 
-  public final static String SCALAR_REPLACEMENT_OPTION =
-      "org.apache.drill.exec.compile.ClassTransformer.scalar_replacement";
-  public final static EnumeratedStringValidator SCALAR_REPLACEMENT_VALIDATOR = new EnumeratedStringValidator(
-      SCALAR_REPLACEMENT_OPTION, null, "try", "off", "try", "on");
-
   @VisibleForTesting // although we need it even if it weren't used in testing
   public enum ScalarReplacementOption {
     OFF, // scalar replacement will not ever be used
@@ -235,7 +230,7 @@ public class ClassTransformer {
       final String entireClass,
       final String materializedClassName) throws ClassTransformationException {
     // unfortunately, this hasn't been set up at construction time, so we have to do it here
-    final ScalarReplacementOption scalarReplacementOption = ScalarReplacementOption.fromString(optionManager.getOption(SCALAR_REPLACEMENT_VALIDATOR));
+    final ScalarReplacementOption scalarReplacementOption = ScalarReplacementOption.fromString(optionManager.getOption(ExecConstants.SCALAR_REPLACEMENT_VALIDATOR));
 
     try {
       final long t1 = System.nanoTime();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index 46eb3f7..9dae286 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -60,7 +60,8 @@ public class PlannerSettings implements Context{
       new OptionDescription("Enable hash aggregation; otherwise, Drill does a sort-based aggregation. Writes to disk. Enable is recommended."));
   public static final OptionValidator STREAMAGG = new BooleanValidator("planner.enable_streamagg",
       new OptionDescription("Sort-based operation. Writes to disk."));
-  public static final OptionValidator TOPN = new BooleanValidator("planner.enable_topn", null);
+  public static final OptionValidator TOPN = new BooleanValidator("planner.enable_topn",
+      new OptionDescription("Generates the topN plan for queries with the ORDER BY and LIMIT clauses."));
   public static final OptionValidator HASHJOIN = new BooleanValidator("planner.enable_hashjoin",
       new OptionDescription("Enable the memory hungry hash join. Drill assumes that a query will have adequate memory to complete and tries to use the fastest operations possible to complete the planned inner, left, right, or full outer joins using a hash table. Does not write to disk. Disabling hash join allows Drill to manage arbitrarily large data in a small memory footprint."));
   public static final OptionValidator MERGEJOIN = new BooleanValidator("planner.enable_mergejoin",
@@ -83,8 +84,8 @@ public class PlannerSettings implements Context{
       new OptionDescription("The factor for adjusting the estimated row count when considering multiple join order sequences during the planning phase."));
   public static final OptionValidator MUX_EXCHANGE = new BooleanValidator("planner.enable_mux_exchange",
       new OptionDescription("Toggles the state of hashing to a multiplexed exchange."));
-  public static final OptionValidator ORDERED_MUX_EXCHANGE = new BooleanValidator("planner.enable_ordered_mux_exchange",
-      null);
+  public static final OptionValidator ORDERED_MUX_EXCHANGE = new BooleanValidator(ExecConstants.ORDERED_MUX_EXCHANGE,
+      new OptionDescription("Generates the MUX exchange operator for ORDER BY queries with many minor fragments."));
   public static final OptionValidator DEMUX_EXCHANGE = new BooleanValidator("planner.enable_demux_exchange",
       new OptionDescription("Toggles the state of hashing to a demulitplexed exchange."));
   public static final OptionValidator PARTITION_SENDER_THREADS_FACTOR = new LongValidator("planner.partitioner_sender_threads_factor",
@@ -130,24 +131,28 @@ public class PlannerSettings implements Context{
 
   public static final String TYPE_INFERENCE_KEY = "planner.enable_type_inference";
   public static final BooleanValidator TYPE_INFERENCE = new BooleanValidator(TYPE_INFERENCE_KEY, null);
-  public static final LongValidator IN_SUBQUERY_THRESHOLD =
-      new PositiveLongValidator("planner.in_subquery_threshold", Integer.MAX_VALUE, null); /* Same as Calcite's default IN List subquery size */
+  public static final LongValidator IN_SUBQUERY_THRESHOLD = new PositiveLongValidator("planner.in_subquery_threshold", Integer.MAX_VALUE,
+      new OptionDescription("Defines the threshold of values in the IN list of the query to generate a hash join instead of an OR predicate.")); /* Same as Calcite's default IN List subquery size */
 
   public static final String PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY = "planner.store.parquet.rowgroup.filter.pushdown.enabled";
-  public static final BooleanValidator PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING = new BooleanValidator(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY, null);
+  public static final BooleanValidator PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING = new BooleanValidator(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_KEY,
+      new OptionDescription("Enables filter pushdown optimization for Parquet files. Drill reads the file metadata, stored in the footer, to eliminate row groups based on the filter condition. Default is true. (Drill 1.9+)"));
   public static final String PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY = "planner.store.parquet.rowgroup.filter.pushdown.threshold";
-  public static final PositiveLongValidator PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD = new PositiveLongValidator(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY,
-      Long.MAX_VALUE, null);
+  public static final PositiveLongValidator PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD = new PositiveLongValidator(PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD_KEY, Long.MAX_VALUE,
+      new OptionDescription("Sets the number of row groups that a table can have. You can increase the threshold if the filter can prune many row groups. However, if this setting is too high, the filter evaluation overhead increases. Base this setting on the data set. Reduce this setting if the planning time is significant or you do not see any benefit at runtime. (Drill 1.9+)"));
 
   public static final String QUOTING_IDENTIFIERS_KEY = "planner.parser.quoting_identifiers";
   public static final EnumeratedStringValidator QUOTING_IDENTIFIERS = new EnumeratedStringValidator(
-      QUOTING_IDENTIFIERS_KEY, null, Quoting.BACK_TICK.string, Quoting.DOUBLE_QUOTE.string, Quoting.BRACKET.string);
+      QUOTING_IDENTIFIERS_KEY,
+      new OptionDescription("Sets the type of identifier quotes for the SQL parser. Default is backticks ('`'). The SQL parser accepts double quotes ('\"') and square brackets ('['). (Drill 1.11+)"),
+      Quoting.BACK_TICK.string, Quoting.DOUBLE_QUOTE.string, Quoting.BRACKET.string);
 
   /*
     "planner.enable_unnest_lateral" is to allow users to choose enable unnest+lateraljoin feature.
    */
   public static final String ENABLE_UNNEST_LATERAL_KEY = "planner.enable_unnest_lateral";
-  public static final BooleanValidator ENABLE_UNNEST_LATERAL = new BooleanValidator(ENABLE_UNNEST_LATERAL_KEY, null);
+  public static final BooleanValidator ENABLE_UNNEST_LATERAL = new BooleanValidator(ENABLE_UNNEST_LATERAL_KEY,
+      new OptionDescription("Enables lateral join functionality. Default is false. (Drill 1.14+)"));
 
   /*
      Enables rules that re-write query joins in the most optimal way.
@@ -171,10 +176,12 @@ public class PlannerSettings implements Context{
      Note: once hash and merge joins will allow non-equi join conditions,
      the need to turn off join optimization may go away.
    */
-  public static final BooleanValidator JOIN_OPTIMIZATION = new BooleanValidator("planner.enable_join_optimization", null);
+  public static final BooleanValidator JOIN_OPTIMIZATION = new BooleanValidator("planner.enable_join_optimization",
+      new OptionDescription("Enables join ordering optimization."));
   // for testing purpose
   public static final String FORCE_2PHASE_AGGR_KEY = "planner.force_2phase_aggr";
-  public static final BooleanValidator FORCE_2PHASE_AGGR = new BooleanValidator(FORCE_2PHASE_AGGR_KEY, null);
+  public static final BooleanValidator FORCE_2PHASE_AGGR = new BooleanValidator(FORCE_2PHASE_AGGR_KEY,
+      new OptionDescription("Forces the cost-based query planner to generate a two phase aggregation for an aggregate operator."));
 
   public OptionManager options = null;
   public FunctionImplementationRegistry functionImplementationRegistry = null;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 1a168c3..d86f67f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -33,7 +33,6 @@ import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.compile.ClassCompilerSelector;
-import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.store.sys.PersistentStore;
 import org.apache.drill.exec.store.sys.PersistentStoreConfig;
@@ -215,7 +214,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       new OptionDefinition(ClassCompilerSelector.JAVA_COMPILER_DEBUG),
       new OptionDefinition(ExecConstants.ENABLE_VERBOSE_ERRORS),
       new OptionDefinition(ExecConstants.ENABLE_WINDOW_FUNCTIONS_VALIDATOR),
-      new OptionDefinition(ClassTransformer.SCALAR_REPLACEMENT_VALIDATOR),
+      new OptionDefinition(ExecConstants.SCALAR_REPLACEMENT_VALIDATOR),
       new OptionDefinition(ExecConstants.ENABLE_NEW_TEXT_READER),
       new OptionDefinition(ExecConstants.ENABLE_BULK_LOAD_TABLE_LIST),
       new OptionDefinition(ExecConstants.BULK_LOAD_TABLE_LIST_BULK_SIZE),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
index 8cab4dd..c02acd2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
@@ -480,6 +480,9 @@ public class WebServer implements AutoCloseable {
     OptionManager optionManager = this.drillbit.getContext().getOptionManager();
     OptionList publicOptions = optionManager.getPublicOptionList();
     List<OptionValue> options = new ArrayList<>(publicOptions);
+    //Add internal options
+    OptionList internalOptions = optionManager.getInternalOptionList();
+    options.addAll(internalOptions);
     Collections.sort(options);
     int numLeftToWrite = options.size();
 
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 66ad9e8..f54655d 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -451,7 +451,6 @@ drill.exec.options: {
     drill.exec.memory.operator.output_batch_size : 16777216, # 16 MB
     drill.exec.memory.operator.output_batch_size_avail_mem_factor : 0.1,
     exec.bulk_load_table_list.bulk_size: 1000,
-    exec.compile.scalar_replacement: false,
     exec.enable_bulk_load_table_list: false,
     exec.enable_union_type: false,
     exec.errors.verbose: false,
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryTestUtil.java
index 1596562..2c89694 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryTestUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryTestUtil.java
@@ -257,8 +257,8 @@ public class QueryTestUtil {
     // set the system option
     final DrillbitContext drillbitContext = drillbit.getContext();
     final SystemOptionManager optionManager = drillbitContext.getOptionManager();
-    final OptionValue originalOptionValue = optionManager.getOption(ClassTransformer.SCALAR_REPLACEMENT_OPTION);
-    optionManager.setLocalOption(ClassTransformer.SCALAR_REPLACEMENT_OPTION, srOption.name().toLowerCase());
+    final OptionValue originalOptionValue = optionManager.getOption(ExecConstants.SCALAR_REPLACEMENT_OPTION);
+    optionManager.setLocalOption(ExecConstants.SCALAR_REPLACEMENT_OPTION, srOption.name().toLowerCase());
 
     // flush the code cache
     drillbitContext.getCompiler().flushCache();
@@ -280,7 +280,7 @@ public class QueryTestUtil {
     final DrillbitContext drillbitContext = drillbit.getContext();
     @SuppressWarnings("resource")
     final OptionManager optionManager = drillbitContext.getOptionManager();
-    optionManager.setLocalOption(ClassTransformer.SCALAR_REPLACEMENT_OPTION, srOption);
+    optionManager.setLocalOption(ExecConstants.SCALAR_REPLACEMENT_OPTION, srOption);
 
     // flush the code cache
     drillbitContext.getCompiler().flushCache();