Configuring CarbonData

This guide explains the configurations that can be used to tune CarbonData to achieve better performance. Most of the properties that control the internal settings have reasonable default values. They are listed along with the properties along with explanation.

System Configuration

This section provides the details of all the configurations required for the CarbonData System.

Property Default Value Description
carbon.ddl.base.hdfs.url (none) To simplify and shorten the path to be specified in DDL/DML commands, this property is supported. This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS of core-site.xml. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv.
carbon.badRecords.location (none) CarbonData can detect the records not conforming to defined table schema and isolate them as bad records. This property is used to specify where to store such bad records.
carbon.streaming.auto.handoff.enabled true CarbonData supports storing of streaming data. To have high throughput for streaming, the data is written in Row format which is highly optimized for write, but performs poorly for query. When this property is true and when the streaming data size reaches carbon.streaming.segment.max.size, CabonData will automatically convert the data to columnar format and optimize it for faster querying. NOTE: It is not recommended to keep the default value which is true.
carbon.streaming.segment.max.size 1024000000 CarbonData writes streaming data in row format which is optimized for high write throughput. This property defines the maximum size of data to be held is row format, beyond which it will be converted to columnar format in order to support high performance query, provided carbon.streaming.auto.handoff.enabled is true. NOTE: Setting higher value will impact the streaming ingestion. The value has to be configured in bytes.
carbon.segment.lock.files.preserve.hours 48 In order to support parallel data loading onto the same table, CarbonData sequences(locks) at the granularity of segments. Operations affecting the segment(like IUD, alter) are blocked from parallel operations. This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours.
carbon.timestamp.format yyyy-MM-dd HH:mm:ss CarbonData can understand data of timestamp type and process it in special manner. It can be so that the format of Timestamp data is different from that understood by CarbonData by default. This configuration allows users to specify the format of Timestamp in their data.
carbon.lock.type LOCALLOCK This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking.
carbon.lock.path TABLEPATH This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.
enable.offheap.sort true Whether carbondata will use offheap or onheap memory. By default, the value is true and carbondata will use the property value from carbon.unsafe.working.memory.in.mb or carbon.unsafe.driver.working.memory.in.mb as the amount of memory; if it is false, carbondata will use the minimum value between the configured amount of unsafe memory and the 60% of JVM Heap Memory as the amount of memory.
carbon.unsafe.working.memory.in.mb 512 CarbonData supports storing data in off-heap memory for certain operations during data loading and query. This helps to avoid the Java GC and thereby improve the overall performance. The Minimum value recommeded is 512MB. Any value below this is reset to default value of 512MB. NOTE: The below formulas explain how to arrive at the off-heap size required.Memory Required For Data Loading per executor: (carbon.number.of.cores.while.loading) * (Number of tables to load in parallel) * (offheap.sort.chunk.size.inmb + carbon.blockletgroup.size.in.mb + carbon.blockletgroup.size.in.mb/3.5 ). Memory required for Query per executor: (carbon.blockletgroup.size.in.mb + carbon.blockletgroup.size.in.mb * 3.5) * spark.executor.cores
carbon.unsafe.driver.working.memory.in.mb (none) CarbonData supports storing data in unsafe on-heap memory in driver for certain operations like insert into, query for loading index cache. The Minimum value recommended is 512MB. If this configuration is not set, carbondata will use the value of carbon.unsafe.working.memory.in.mb.
carbon.update.sync.folder /tmp/carbondata CarbonData maintains last modification time entries in modifiedTime.htmlt to determine the schema changes and reload only when necessary. This configuration specifies the path where the file needs to be written.
carbon.invisible.segments.preserve.count 200 CarbonData maintains each data load entry in tablestatus file. The entries from this file are not deleted for those segments that are compacted or dropped, but are made invisible. If the number of data loads are very high, the size and number of entries in tablestatus file can become too many causing unnecessary reading of all data. This configuration specifies the number of segment entries to be maintained afte they are compacted or dropped. Beyond this, the entries are moved to a separate history tablestatus file. NOTE: The entries in tablestatus file help to identify the operations performed on CarbonData table and is also used for checkpointing during various data manupulation operations. This is similar to AUDIT file maintaining all the operations and its status. Hence the entries are never deleted but moved to a separate history file.
carbon.lock.retries 3 CarbonData ensures consistency of operations by blocking certain operations from running in parallel. In order to block the operations from running in parallel, lock is obtained on the table. This configuration specifies the maximum number of retries to obtain the lock for any operations other than load. NOTE: Data manupulation operations like Compaction,UPDATE,DELETE or LOADING,UPDATE,DELETE are not allowed to run in parallel. How ever data loading can happen in parallel to compaction.
carbon.lock.retry.timeout.sec 5 Specifies the interval between the retries to obtain the lock for any operation other than load. NOTE: Refer to carbon.lock.retries for understanding why CarbonData uses locks for operations.
carbon.fs.custom.file.provider None To support FileTypeInterface for configuring custom CarbonFile implementation to work with custom FileSystem.
carbon.timeseries.first.day.of.week SUNDAY This parameter configures which day of the week to be considered as first day of the week. Because first day of the week will be different in different parts of the world.
carbon.enable.tablestatus.backup false In cloud object store scenario, overwriting table status file is not an atomic operation since it uses rename API. Thus, it is possible that table status is corrupted if process crashed when overwriting the table status file. To protect from file corruption, user can enable this property.

Data Loading Configuration

Parameter Default Value Description
carbon.concurrent.lock.retries 100 CarbonData supports concurrent data loading onto same table. To ensure the loading status is correctly updated into the system,locks are used to sequence the status updation step. This configuration specifies the maximum number of retries to obtain the lock for updating the load status. NOTE: This value is high as more number of concurrent loading happens,more the chances of not able to obtain the lock when tried. Adjust this value according to the number of concurrent loading to be supported by the system.
carbon.concurrent.lock.retry.timeout.sec 1 Specifies the interval between the retries to obtain the lock for concurrent operations. NOTE: Refer to carbon.concurrent.lock.retries for understanding why CarbonData uses locks during data loading operations.
carbon.csv.read.buffersize.byte 1048576 CarbonData uses Hadoop InputFormat to read the csv files. This configuration value is used to pass buffer size as input for the Hadoop MR job when reading the csv files. This value is configured in bytes. NOTE: Refer to org.apache.hadoop.mapreduce. InputFormat documentation for additional information.
carbon.loading.prefetch false CarbonData uses univocity parser to read csv files. This configuration is used to inform the parser whether it can prefetch the data from csv files to speed up the reading. NOTE: Enabling prefetch improves the data loading performance, but needs higher memory to keep more records which are read ahead from disk.
carbon.skip.empty.line false The csv files givent to CarbonData for loading can contain empty lines. Based on the business scenario, this empty line might have to be ignored or needs to be treated as NULL value for all columns. In order to define this business behavior, this configuration is provided. NOTE: In order to consider NULL values for non string columns and continue with data load, carbon.bad.records.action need to be set to FORCE;else data load will be failed as bad records encountered.
carbon.number.of.cores.while.loading 2 Number of cores to be used while loading data. This also determines the number of threads to be used to read the input files (csv) in parallel. NOTE: This configured value is used in every data loading step to parallelize the operations. Configuring a higher value can lead to increased early thread pre-emption by OS and there by reduce the overall performance.
enable.unsafe.sort true CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. This configuration enables to use unsafe functions in CarbonData. NOTE: For operations like data loading, which generates more short lived Java objects, Java GC can be a bottle neck. Using unsafe can overcome the GC overhead and improve the overall performance.
enable.offheap.sort true CarbonData supports storing data in off-heap memory for certain operations during data loading and query. This helps to avoid the Java GC and thereby improve the overall performance. This configuration enables using off-heap memory for sorting of data during data loading.NOTE: enable.unsafe.sort configuration needs to be configured to true for using off-heap
carbon.load.sort.scope NO_SORT [If sort columns are not specified while creating table] and LOCAL_SORT [If sort columns are specified] CarbonData can support various sorting options to match the balance between load and query performance. LOCAL_SORT: All the data given to an executor in the single load is fully sorted and written to carbondata files. Data loading performance is reduced a little as the entire data needs to be sorted in the executor. GLOBAL SORT: Entire data in the data load is fully sorted and written to carbondata files. Data loading performance would get reduced as the entire data needs to be sorted. But the query performance increases significantly due to very less false positives and concurrency is also improved. NOTE 1: This property will be taken into account only when SORT COLUMNS are specified explicitly while creating table, otherwise it is always NO SORT
carbon.global.sort.rdd.storage.level MEMORY_ONLY Storage level to persist dataset of RDD/dataframe when loading data with 'sort_scope'='global_sort', if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. See detail.
carbon.load.global.sort.partitions 0 The number of partitions to use when shuffling data for global sort. Default value 0 means to use same number of map tasks as reduce tasks. NOTE: In general, it is recommended to have 2-3 tasks per CPU core in your cluster.
carbon.sort.size 100000 Number of records to hold in memory to sort and write intermediate sort temp files. NOTE: Memory required for data loading will increase if you turn this value bigger. Besides each thread will cache this amout of records. The number of threads is configured by carbon.number.of.cores.while.loading.
carbon.options.bad.records.logger.enable false CarbonData can identify the records that are not conformant to schema and isolate them as bad records. Enabling this configuration will make CarbonData to log such bad records. NOTE: If the input data contains many bad records, logging them will slow down the over all data loading throughput. The data load operation status would depend on the configuration in carbon.bad.records.action.
carbon.bad.records.action FAIL CarbonData in addition to identifying the bad records, can take certain actions on such data. This configuration can have four types of actions for bad records namely FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found.
carbon.options.is.empty.data.bad.record false Based on the business scenarios, empty("" or '' or ,,) data can be valid or invalid. This configuration controls how empty data should be treated by CarbonData. If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.
carbon.options.bad.record.path (none) Specifies the HDFS path where bad records are to be stored. By default the value is Null. This path must be configured by the user if carbon.options.bad.records.logger.enable is true or carbon.bad.records.action is REDIRECT.
carbon.blockletgroup.size.in.mb 64 Please refer to file-structure-of-carbondata to understand the storage format of CarbonData. The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access. The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB). NOTE: Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing. For filter queries with limit, it is not advisable to have a bigger blocklet size. For aggregation queries which need to return more number of rows, bigger blocklet size is advisable.
carbon.sort.file.write.buffer.size 16384 CarbonData sorts and writes data to intermediate files to limit the memory usage. This configuration determines the buffer size to be used for reading and writing such files. NOTE: This configuration is useful to tune IO and derive optimal performance. Based on the OS and underlying harddisk type, these values can significantly affect the overall performance. It is ideal to tune the buffer size equivalent to the IO buffer size of the OS. Recommended range is between 10240 and 10485760 bytes.
carbon.sort.intermediate.files.limit 20 CarbonData sorts and writes data to intermediate files to limit the memory usage. Before writing the target carbondata file, the records in these intermediate files needs to be merged to reduce the number of intermediate files. This configuration determines the minimum number of intermediate files after which merged sort is applied on them sort the data. NOTE: Intermediate merging happens on a separate thread in the background. Number of threads used is determined by carbon.merge.sort.reader.thread. Configuring a low value will cause more time to be spent in merging these intermediate merged files which can cause more IO. Configuring a high value would cause not to use the idle threads to do intermediate sort merges. Recommended range is between 2 and 50.
carbon.merge.sort.reader.thread 3 CarbonData sorts and writes data to intermediate files to limit the memory usage. When the intermediate files reaches carbon.sort.intermediate.files.limit, the files will be merged in another thread pool. This value will control the size of the pool. Each thread will read the intermediate files and do merge sort and finally write the records to another file. NOTE: Refer to carbon.sort.intermediate.files.limit for operation description. Configuring smaller number of threads can cause merging slow down over loading process whereas configuring larger number of threads can cause thread contention with threads in other data loading steps. Hence configure a fraction of carbon.number.of.cores.while.loading.
carbon.merge.sort.prefetch true CarbonData writes every carbon.sort.size number of records to intermediate temp files during data loading to ensure memory footprint is within limits. These intermediate temp files will have to be sorted using merge sort before writing into CarbonData format. This configuration enables pre fetching of data from these temp files in order to optimize IO and speed up data loading process.
carbon.prefetch.buffersize 1000 When the configuration carbon.merge.sort.prefetch is configured to true, we need to set the number of records that can be prefetched. This configuration is used specify the number of records to be prefetched.**NOTE: **Configuring more number of records to be prefetched increases memory footprint as more records will have to be kept in memory.
carbon.sort.storage.inmemory.size.inmb 512 CarbonData writes every carbon.sort.size number of records to intermediate temp files during data loading to ensure memory footprint is within limits. When enable.unsafe.sort configuration is enabled, instead of using carbon.sort.size which is based on rows count, size occupied in memory is used to determine when to flush data pages to intermediate temp files. This configuration determines the memory to be used for storing data pages in memory. NOTE: Configuring a higher value ensures more data is maintained in memory and hence increases data loading performance due to reduced or no IO. Based on the memory availability in the nodes of the cluster, configure the values accordingly.
carbon.load.sortmemory.spill.percentage 0 During data loading, some data pages are kept in memory upto memory configured in carbon.sort.storage.inmemory.size.inmb beyond which they are spilled to disk as intermediate temporary sort files. This configuration determines after what percentage data needs to be spilled to disk. NOTE: Without this configuration, when the data pages occupy upto configured memory, new data pages would be dumped to disk and old pages are still maintained in disk.
carbon.enable.calculate.size true For Load Operation: Enabling this property will let carbondata calculate the size of the carbon data file (.carbondata) and the carbon index file (.carbonindex) for each load and update the table status file. For Describe Formatted: Enabling this property will let carbondata calculate the total size of the carbon data files and the carbon index files for the each table and display it in describe formatted command. NOTE: This is useful to determine the overall size of the carbondata table and also get an idea of how the table is growing in order to take up other backup strategy decisions.
carbon.cutOffTimestamp (none) CarbonData has capability to generate the Dictionary values for the timestamp columns from the data itself without the need to store the computed dictionary values. This configuration sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". NOTE: The date must be in the form carbon.timestamp.format. CarbonData supports storing data for upto 68 years. For example, if the cut-off time is 1970-01-01 05:30:00, then data upto 2038-01-01 05:30:00 will be supported by CarbonData.
carbon.timegranularity SECOND The configuration is used to specify the data granularity level such as DAY, HOUR, MINUTE, or SECOND. This helps to store more than 68 years of data into CarbonData.
carbon.use.local.dir true CarbonData,during data loading, writes files to local temp directories before copying the files to HDFS. This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory.
carbon.sort.temp.compressor SNAPPY CarbonData writes every carbon.sort.size number of records to intermediate temp files during data loading to ensure memory footprint is within limits. These temporary files can be compressed and written in order to save the storage space. This configuration specifies the name of compressor to be used to compress the intermediate sort temp files during sort procedure in data loading. The valid values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. NOTE: Compressor will be useful if you encounter disk bottleneck. Since the data needs to be compressed and decompressed,it involves additional CPU cycles,but is compensated by the high IO throughput due to less data to be written or read from the disks.
carbon.load.skewedDataOptimization.enabled false During data loading,CarbonData would divide the number of blocks equally so as to ensure all executors process same number of blocks. This mechanism satisfies most of the scenarios and ensures maximum parallel processing for optimal data loading performance. In some business scenarios, there might be scenarios where the size of blocks vary significantly and hence some executors would have to do more work if they get blocks containing more data. This configuration enables size based block allocation strategy for data loading. When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data. NOTE: This configuration is useful if the size of your input data files varies widely, say 1MB to 1GB. For this configuration to work effectively,knowing the data pattern and size is important and necessary.
enable.data.loading.statistics false CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues. This configuration when made true would log additional data loading statistics information to more accurately locate the issues being debugged. NOTE: Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time. It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately. Also extensive logging is an increased IO operation and hence over all data loading performance might get reduced. Therefore it is recommended to enable this configuration only for the duration of debugging.
carbon.dictionary.chunk.size 10000 CarbonData generates dictionary keys and writes them to separate dictionary file during data loading. To optimize the IO, this configuration determines the number of dictionary keys to be persisted to dictionary file at a time. NOTE: Writing to file also serves as a commit point to the dictionary generated. Increasing more values in memory causes more data loss during system or application failure. It is advised to alter this configuration judiciously.
carbon.load.directWriteToStorePath.enabled false During data load, all the carbondata files are written to local disk and finally copied to the target store location in HDFS/S3. Enabling this parameter will make carbondata files to be written directly onto target HDFS/S3 location bypassing the local disk. NOTE: Writing directly to HDFS/S3 saves local disk IO(once for writing the files and again for copying to HDFS/S3) there by improving the performance. But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS/S3 location until it is cleared during next data load or by running CLEAN FILES DDL command
carbon.options.serialization.null.format \N Based on the business scenarios, some columns might need to be loaded with null values. As null value cannot be written in csv files, some special characters might be adopted to specify null values. This configuration can be used to specify the null values format in the data being loaded.
carbon.column.compressor snappy CarbonData will compress the column values using the compressor specified by this configuration. Currently CarbonData supports 'snappy', 'zstd' and 'gzip' compressors.
carbon.minmax.allowed.byte.count 200 CarbonData will write the min max values for string/varchar types column using the byte count specified by this configuration. Max value is 1000 bytes(500 characters) and Min value is 10 bytes(5 characters). NOTE: This property is useful for reducing the store size thereby improving the query performance but can lead to query degradation if value is not configured properly.
carbon.merge.index.failure.throw.exception true It is used to configure whether or not merge index failure should result in data load failure also.
carbon.binary.decoder None Support configurable decode for loading. Two decoders supported: base64 and hex
carbon.local.dictionary.size.threshold.inmb 4 size based threshold for local dictionary in MB, maximum allowed size is 16 MB.
carbon.enable.bad.record.handling.for.insert false by default, disable the bad record and converter step during "insert into"

Compaction Configuration

Parameter Default Value Description
carbon.number.of.cores.while.compacting 2 Number of cores to be used while compacting data. This also determines the number of threads to be used to read carbondata files in parallel.
carbon.compaction.level.threshold 4, 3 Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. This configuration is for minor compaction which decides how many segments to be merged. Configuration is of the form (x,y). Compaction will be triggered for every x segments and form a single level 1 compacted segment. When the number of compacted level 1 segments reach y, compaction will be triggered again to merge them to form a single level 2 segment. For example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segments which is further compacted to new segment. NOTE: When carbon.enable.auto.load.merge is true, configuring higher values cause overall data loading time to increase as compaction will be triggered after data loading is complete but status is not returned till compaction is complete. But compacting more number of segments can increase query performance. Hence optimal values needs to be configured based on the business scenario. Valid values are between 0 to 100.
carbon.major.compaction.size 1024 To improve query performance and all the segments can be merged and compacted to a single segment upto configured size. This Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB.
carbon.horizontal.compaction.enable true CarbonData supports DELETE/UPDATE functionality by creating delta data files for existing carbondata files. These delta files would grow as more number of DELETE/UPDATE operations are performed. Compaction of these delta files are termed as horizontal compaction. This configuration is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold. NOTE: Having many delta files will reduce the query performance as scan has to happen on all these files before the final state of data can be decided. Hence it is advisable to keep horizontal compaction enabled and configure reasonable values to carbon.horizontal.DELETE.compaction.threshold
carbon.horizontal.delete.compaction.threshold 1 This configuration specifies the threshold limit on number of DELETE delta files within a block of a segment. In case the number of delta files goes beyond the threshold, the DELETE delta files for the particular block of the segment becomes eligible for horizontal compaction and are compacted into single DELETE delta file. Values range between 1 to 10000.
carbon.update.segment.parallelism 1 CarbonData processes the UPDATE operations by grouping records belonging to a segment into a single executor task. When the amount of data to be updated is more, this behavior causes problems like restarting of executor due to low memory and data-spill related errors. This property specifies the parallelism for each segment during update. NOTE: It is recommended to set this value to a multiple of the number of executors for balance. Values range between 1 to 1000.
carbon.numberof.preserve.segments 0 If the user wants to preserve some number of segments from being compacted then he can set this configuration. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default. NOTE: This configuration is useful when the chances of input data can be wrong due to environment scenarios. Preserving some of the latest segments from being compacted can help to easily delete the wrongly loaded segments. Once compacted,it becomes more difficult to determine the exact data to be deleted(except when data is incrementing according to time)
carbon.allowed.compaction.days 0 This configuration is used to control on the number of recent segments that needs to be compacted, ignoring the older ones. This configuration is in days. For Example: If the configuration is 2, then the segments which are loaded in the time frame of past 2 days only will get merged. Segments which are loaded earlier than 2 days will not be merged. This configuration is disabled by default. NOTE: This configuration is useful when a bulk of history data is loaded into the carbondata. Query on this data is less frequent. In such cases involving these segments also into compaction will affect the resource consumption, increases overall compaction time.
carbon.enable.auto.load.merge false Compaction can be automatically triggered once data load completes. This ensures that the segments are merged in time and thus query times does not increase with increase in segments. This configuration enables to do compaction along with data loading. NOTE: Compaction will be triggered once the data load completes. But the status of data load wait till the compaction is completed. Hence it might look like data loading time has increased, but thats not the case. Moreover failure of compaction will not affect the data loading status. If data load had completed successfully, the status would be updated and segments are committed. However, failure while data loading, will not trigger compaction and error is returned immediately.
carbon.enable.page.level.reader.in.compaction false Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory. NOTE: Please refer to file-structure-of-carbondata to understand the storage format of CarbonData and concepts of pages.
carbon.concurrent.compaction true Compaction of different tables can be executed concurrently. This configuration determines whether to compact all qualifying tables in parallel or not. NOTE: Compacting concurrently is a resource demanding operation and needs more resources there by affecting the query performance also. This configuration is deprecated and might be removed in future releases.
carbon.compaction.prefetch.enable false Compaction operation is similar to Query + data load where in data from qualifying segments are queried and data loading performed to generate a new single segment. This configuration determines whether to query ahead data from segments and feed it for data loading. NOTE: This configuration is disabled by default as it needs extra resources for querying extra data. Based on the memory availability on the cluster, user can enable it to improve compaction performance.
carbon.merge.index.in.segment true Each CarbonData file has a companion CarbonIndex file which maintains the metadata about the data. These CarbonIndex files are read and loaded into driver and is used subsequently for pruning of data during queries. These CarbonIndex files are very small in size(few KB) and are many. Reading many small files from HDFS is not efficient and leads to slow IO performance. Hence these CarbonIndex files belonging to a segment can be combined into a single file and read once there by increasing the IO throughput. This configuration enables to merge all the CarbonIndex files into a single MergeIndex file upon data loading completion. NOTE: Reading a single big file is more efficient in HDFS and IO throughput is very high. Due to this the time needed to load the index files into memory when query is received for the first time on that table is significantly reduced and there by significantly reduces the delay in serving the first query.
carbon.enable.range.compaction true To configure Ranges-based Compaction to be used or not for RANGE_COLUMN. If true after compaction also the data would be present in ranges.
carbon.si.segment.merge false Making this true degrade the LOAD performance. When the number of small files increase for SI segments(it can happen as number of columns will be less and we store position id and reference columns), user an either set to true which will merge the data files for upcoming loads or run SI refresh command which does this job for all segments. (REFRESH INDEX <index_table>)

Query Configuration

Parameter Default Value Description
carbon.max.driver.lru.cache.size -1 Maximum memory (in MB) upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. NOTE: Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others. Please refer FAQs for checking LRU cache memory footprint.
carbon.max.executor.lru.cache.size -1 Maximum memory (in MB) upto which the executor process can cache the data (BTree and reverse dictionary values). Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. NOTE: If this parameter is not configured, then the value of carbon.max.driver.lru.cache.size will be used.
max.query.execution.time 60 Maximum time allowed for one query to be executed. The value is in minutes.
carbon.enableMinMax true CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions. To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file. This pruning enhances query performance significantly.
carbon.dynamical.location.scheduler.timeout 5 CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData. To determine the number of tasks that can be scheduled, knowing the count of active executors is necessary. When dynamic allocation is enabled on a YARN based spark cluster, executor processes are shutdown if no request is received for a particular amount of time. The executors are brought up when the requet is received again. This configuration specifies the maximum time (unit in seconds) the carbon scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.NOTE: Waiting for longer time leads to slow query response time.Moreover it might be possible that YARN is not able to start the executors and waiting is not beneficial.
carbon.scheduler.min.registered.resources.ratio 0.8 Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution. The minimum value is 0.1 min and the maximum value is 1.0.
carbon.detail.batch.size 100 The buffer size to store records, returned from the block scan. In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000.

NOTE The minimum batch size allowed is 100 and maximum batch size allowed by this property is 1000.
carbon.enable.vector.reader true Spark added vector processing to optimize cpu cache miss and there by increase the query performance. This configuration enables to fetch data as columnar batch of size 4*1024 rows instead of fetching data row by row and provide it to spark so that there is improvement in select queries performance.
carbon.task.distribution block CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData. Each of these task distribution suggestions has its own advantages and disadvantages. Based on the customer use case, appropriate task distribution can be configured.block: Setting this value will launch one task per block. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. custom: Setting this value will group the blocks and distribute it uniformly to the available resources in the cluster. This enhances the query performance but not suggested in case of concurrent queries and queries having big shuffling scenarios. blocklet: Setting this value will launch one task per blocklet. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. merge_small_files: Setting this value will merge all the small carbondata files upto a bigger size configured by spark.sql.files.maxPartitionBytes (128 MB is the default value,it is configurable) during querying. The small carbondata files are combined to a map task to reduce the number of read task. This enhances the performance.
carbon.custom.block.distribution false CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData. When this configuration is true, CarbonData would distribute the available blocks to be scanned among the available number of cores. For Example:If there are 10 blocks to be scanned and only 3 tasks can be run(only 3 executor cores available in the cluster), CarbonData would combine blocks as 4,3,3 and give it to 3 tasks to run. NOTE: When this configuration is false, as per the carbon.task.distribution configuration, each block/blocklet would be given to each task.
enable.query.statistics false CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues. This configuration when made true would log additional query statistics information to more accurately locate the issues being debugged. NOTE: Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time. It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately. Also extensive logging is an increased IO operation and hence over all query performance might get reduced. Therefore it is recommended to enable this configuration only for the duration of debugging.
enable.unsafe.in.query.processing false CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. This configuration enables to use unsafe functions in CarbonData while scanning the data during query.
carbon.max.driver.threads.for.block.pruning 4 Number of threads used for driver pruning when the carbon files are more than 100k Maximum memory. This configuration can used to set number of threads between 1 to 4.
carbon.heap.memory.pooling.threshold.bytes 1048576 CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP. If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism. But if set this size to -1, it should not go through the pooling mechanism. Default value is 1048576(1MB, the same as Spark). Value to be specified in bytes.
carbon.push.rowfilters.for.vector false When enabled complete row filters will be handled by carbon in case of vector. If it is disabled then only page level pruning will be done by carbon and row level filtering will be done by spark for vector. And also there are scan optimizations in carbon to avoid multiple data copies when this parameter is set to false. There is no change in flow for non-vector based queries.
carbon.query.prefetch.enable true By default this property is true, so prefetch is used in query to read next blocklet asynchronously in other thread while processing current blocklet in main thread. This can help to reduce CPU idle time. Setting this property false will disable this prefetch feature in query.
carbon.query.stage.input.enable false Stage input files are data files written by external applications (such as Flink), but have not been loaded into carbon table. Enabling this configuration makes query to include these files, thus makes query on latest data. However, since these files are not indexed, query maybe slower as full scan is required for these files.
carbon.driver.pruning.multi.thread.enable.files.count 100000 To prune in multi-thread when total number of segment files for a query increases beyond the configured value.
carbon.load.all.segment.indexes.to.cache true Setting this configuration to false, will prune and load only matched segment indexes to cache using segment metadata information such as columnid and it's minmax values, which decreases the usage of driver memory.
carbon.secondary.index.creation.threads 1 Specifies the number of threads to concurrently process segments during secondary index creation. This property helps fine tuning the system when there are a lot of segments in a table. The value range is 1 to 50.
carbon.si.lookup.partialstring true When true, it includes starts with, ends with and contains. When false, it includes only starts with secondary indexes.

Data Mutation Configuration

Parameter Default Value Description
carbon.update.persist.enable true Configuration to enable the dataset of RDD/dataframe to persist data. Enabling this will reduce the execution time of UPDATE operation.
carbon.update.storage.level MEMORY_AND_DISK Storage level to persist dataset of a RDD/dataframe. Applicable when carbon.update.persist.enable is true, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. See detail.
carbon.update.check.unique.value true By default this property is true, so update will validate key value mapping. This validation might have slight degrade in performance of update query. If user knows that key value mapping is correct, can disable this validation for better update performance by setting this property to false.

Dynamic Configuration In CarbonData Using SET-RESET

SET/RESET commands are used to add, update, display, or reset the carbondata properties dynamically without restarting the driver.

Syntax

  • Add or Update : This command adds or updates the value of parameter_name.
SET parameter_name=parameter_value
  • Display Property Value: This command displays the value of the specified parameter_name.
SET parameter_name
  • Display Session Parameters: This command displays all the supported session parameters.
SET
  • Display Session Parameters along with usage details: This command displays all the supported session parameters along with their usage details.
SET -v
  • Reset: This command clears all the session parameters.
RESET

Parameter Description:

Parameter Description
parameter_name Name of the property whose value needs to be dynamically added, updated, or displayed.
parameter_value New value of the parameter_name to be set.

Dynamically Configurable Properties of CarbonData

Properties Description
carbon.options.bad.records.logger.enable To enable or disable a bad record logger. CarbonData can identify the records that are not conformant to schema and isolate them as bad records. Enabling this configuration will make CarbonData to log such bad records. NOTE: If the input data contains many bad records, logging them will slow down the overall data loading throughput. The data load operation status would depend on the configuration in carbon.bad.records.action.
carbon.options.bad.records.action This property has four types of bad record actions: FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found.
carbon.options.is.empty.data.bad.record If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.
carbon.options.bad.record.path Specifies the HDFS path where bad records needs to be stored.
carbon.custom.block.distribution Specifies whether to use the Spark or Carbon block distribution feature. NOTE: Refer to Query Configuration#carbon.custom.block.distribution for more details on CarbonData scheduler.
enable.unsafe.sort Specifies whether to use unsafe sort during data loading. Unsafe sort reduces the garbage collection during data load operation, resulting in better performance.
carbon.options.date.format Specifies the data format of the date columns in the data being loaded
carbon.options.timestamp.format Specifies the timestamp format of the time stamp columns in the data being loaded
carbon.options.sort.scope Specifies how the current data load should be sorted with. This sort parameter is at the table level. NOTE: Refer to Data Loading Configuration#carbon.sort.scope for detailed information.
carbon.table.load.sort.scope.<db_name>.<table_name> Overrides the SORT_SCOPE provided in CREATE TABLE.
carbon.options.global.sort.partitions Specifies the number of partitions to be used during global sort.
carbon.options.serialization.null.format Default Null value representation in the data being loaded. NOTE: Refer to Data Loading Configuration#carbon.options.serialization.null.format for detailed information.
carbon.number.of.cores.while.loading Specifies number of cores to be used while loading data. NOTE: Refer to Data Loading Configuration#carbon.number.of.cores.while.loading for detailed information.
carbon.number.of.cores.while.compacting Specifies number of cores to be used while compacting data. NOTE: Refer to Data Loading Configuration#carbon.number.of.cores.while.compacting for detailed information.
enable.offheap.sort To enable off-heap memory usage. NOTE: Refer to Data Loading Configuration#enable.offheap.sort for detailed information.
carbon.blockletgroup.size.in.mb Specifies the size of each blocklet group. NOTE: Refer to Data Loading Configuration#carbon.blockletgroup.size.in.mb for detailed information.
carbon.enable.auto.load.merge To enable compaction along with data loading. NOTE: Refer to Compaction Configuration#carbon.enable.auto.load.merge for detailed information.
carbon.major.compaction.size To configure major compaction size. NOTE: Refer to Compaction Configuration#carbon.major.compaction.size for detailed information.
carbon.compaction.level.threshold To configure compaction threshold. NOTE: Refer to Compaction Configuration#carbon.compaction.level.threshold for detailed information.
carbon.enable.vector.reader To enable fetching data as columnar batch of size 4*1024 rows instead of fetching a row at a time. NOTE: Refer to Query Configuration#carbon.enable.vector.reader for detailed information.
enable.unsafe.in.query.processing To enable use of unsafe functions while scanning the data during query. NOTE: Refer to Query Configuration#enable.unsafe.in.query.processing for detailed information.
carbon.push.rowfilters.for.vector To enable complete row filters handling by carbon in case of vector. NOTE: Refer to Query Configuration#carbon.push.rowfilters.for.vector for detailed information.
carbon.query.stage.input.enable To make query to include staged input files. NOTE: Refer to Query Configuration#carbon.query.stage.input.enable for detailed information.
carbon.input.segments.<db_name>.<table_name> To specify the segment ids to query from the table. segments ids are separated by comma.
carbon.index.visible.<db_name>.<table_name>.<index_name> To specify query on db_name.table_name to not use the index index_name.
carbon.load.indexes.parallel.<db_name>.<table_name> To enable parallel index loading for a table. when db_name.table_name are not specified, i.e., when carbon.load.indexes.parallel. is set, it applies for all the tables of the session.
carbon.enable.index.server To use index server for caching and pruning. This property can be used for a session or for a particular table with carbon.enable.index.server.<db_name>.<table_name>.

Examples:

  • Add or Update:
SET enable.unsafe.sort =true
  • Display Property Value:
SET enable.unsafe.sort
  • Reset:
RESET

System Response:

  • Success will be recorded in the driver log.

  • Failure will be displayed in the UI.