Skip to content

Commit

Permalink
'maxBytesInMemory' tuningConfig introduced for ingestion tasks (apach…
Browse files Browse the repository at this point in the history
…e#5583)

* This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks

Currently a config called 'maxRowsInMemory' is present which affects how much memory gets
used for indexing.If this value is not optimal for your JVM heap size, it could lead
to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might
be bad for query performance and a higher value will limit number of persists but require
more jvm heap space and could lead to OOM.
'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes
kept in memory before persisting.

 * The default value is 1/3(Runtime.maxMemory())
 * To maintain the current behaviour set 'maxBytesInMemory' to -1
 * If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them
   will be respected i.e. the first one to go above threshold will trigger persist

* Fix check style and remove a comment

* Add overlord unsecured paths to coordinator when using combined service (apache#5579)

* Add overlord unsecured paths to coordinator when using combined service

* PR comment

* More error reporting and stats for ingestion tasks (apache#5418)

* Add more indexing task status and error reporting

* PR comments, add support in AppenderatorDriverRealtimeIndexTask

* Use TaskReport instead of metrics/context

* Fix tests

* Use TaskReport uploads

* Refactor fire department metrics retrieval

* Refactor input row serde in hadoop task

* Refactor hadoop task loader names

* Truncate error message in TaskStatus, add errorMsg to task report

* PR comments

* Allow getDomain to return disjointed intervals (apache#5570)

* Allow getDomain to return disjointed intervals

* Indentation issues

* Adding feature thetaSketchConstant to do some set operation in PostAgg (apache#5551)

* Adding feature thetaSketchConstant to do some set operation in PostAggregator

* Updated review comments for PR apache#5551 - Adding thetaSketchConstant

* Fixed CI build issue

* Updated review comments 2 for PR apache#5551 - Adding thetaSketchConstant

* Fix taskDuration docs for KafkaIndexingService (apache#5572)

* With incremental handoff the changed line is no longer true.

* Add doc for automatic pendingSegments (apache#5565)

* Add missing doc for automatic pendingSegments

* address comments

* Fix indexTask to respect forceExtendableShardSpecs (apache#5509)

* Fix indexTask to respect forceExtendableShardSpecs

* add comments

* Deprecate spark2 profile in pom.xml (apache#5581)

Deprecated due to apache#5382

* CompressionUtils: Add support for decompressing xz, bz2, zip. (apache#5586)

Also switch various firehoses to the new method.

Fixes apache#5585.

* This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks

Currently a config called 'maxRowsInMemory' is present which affects how much memory gets
used for indexing.If this value is not optimal for your JVM heap size, it could lead
to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might
be bad for query performance and a higher value will limit number of persists but require
more jvm heap space and could lead to OOM.
'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes
kept in memory before persisting.

 * The default value is 1/3(Runtime.maxMemory())
 * To maintain the current behaviour set 'maxBytesInMemory' to -1
 * If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them
   will be respected i.e. the first one to go above threshold will trigger persist

* Address code review comments

* Fix the coding style according to druid conventions
* Add more javadocs
* Rename some variables/methods
* Other minor issues

* Address more code review comments

* Some refactoring to put defaults in IndexTaskUtils
* Added check for maxBytesInMemory in AppenderatorImpl
* Decrement bytes in abandonSegment
* Test unit test for multiple sinks in single appenderator
* Fix some merge conflicts after rebase

* Fix some style checks

* Merge conflicts

* Fix failing tests

Add back check for 0 maxBytesInMemory in OnHeapIncrementalIndex

* Address PR comments

* Put defaults for maxRows and maxBytes in TuningConfig
* Change/add javadocs
* Refactoring and renaming some variables/methods

* Fix TeamCity inspection warnings

* Added maxBytesInMemory config to HadoopTuningConfig

* Updated the docs and examples

* Added maxBytesInMemory config in docs
* Removed references to maxRowsInMemory under tuningConfig in examples

* Set maxBytesInMemory to 0 until used

Set the maxBytesInMemory to 0 if user does not set it as part of tuningConfing
and set to part of max jvm memory when ingestion task starts

* Update toString in KafkaSupervisorTuningConfig

* Use correct maxBytesInMemory value in AppenderatorImpl

* Update DEFAULT_MAX_BYTES_IN_MEMORY to 1/6 max jvm memory

Experimenting with various defaults, 1/3 jvm memory causes OOM

* Update docs to correct maxBytesInMemory default value

* Minor to rename and add comment

* Add more details in docs

* Address new PR comments

* Address PR comments

* Fix spelling typo
  • Loading branch information
Surekha authored and gianm committed May 3, 2018
1 parent 739e347 commit 13c616b
Show file tree
Hide file tree
Showing 64 changed files with 778 additions and 73 deletions.
3 changes: 2 additions & 1 deletion docs/content/development/extensions-core/kafka-ingestion.md
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|`type`|String|The indexing task type, this should always be `kafka`.|yes|
|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)|
|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists). Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|no (default == 1000000)|
|`maxBytesInMemory`|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally this is computed internally and user does not need to set it. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists). |no (default == One-sixth of max JVM memory)|
|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == 5000000)|
|`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)|
|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)|
Expand Down
3 changes: 2 additions & 1 deletion docs/content/ingestion/batch-ingestion.md
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|workingPath|String|The working path to use for intermediate results (results between Hadoop jobs).|no (default == '/tmp/druid-indexing')|
|version|String|The version of created segments. Ignored for HadoopIndexTask unless useExplicitVersion is set to true|no (default == datetime that indexing starts at)|
|partitionsSpec|Object|A specification of how to partition each time bucket into segments. Absence of this property means no partitioning will occur. See 'Partitioning specification' below.|no (default == 'hashed')|
|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. Note that this is the number of post-aggregation rows which may not be equal to the number of input events due to roll-up. This is used to manage the required JVM heap size.|no (default == 75000)|
|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. Note that this is the number of post-aggregation rows which may not be equal to the number of input events due to roll-up. This is used to manage the required JVM heap size. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|no (default == 1000000)|
|maxBytesInMemory|Long|The number of bytes to aggregate in heap memory before persisting. Normally this is computed internally and user does not need to set it. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists).|no (default == One-sixth of max JVM memory)|
|leaveIntermediate|Boolean|Leave behind intermediate files (for debugging) in the workingPath when a job completes, whether it passes or fails.|no (default == false)|
|cleanupOnFailure|Boolean|Clean up intermediate files when a job fails (unless leaveIntermediate is on).|no (default == true)|
|overwriteFiles|Boolean|Override existing files found during indexing.|no (default == false)|
Expand Down
8 changes: 5 additions & 3 deletions docs/content/ingestion/stream-pull.md
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ The property `druid.realtime.specFile` has the path of a file (absolute or relat
},
"tuningConfig": {
"type" : "realtime",
"maxRowsInMemory": 75000,
"maxRowsInMemory": 1000000,
"intermediatePersistPeriod": "PT10m",
"windowPeriod": "PT10m",
"basePersistDirectory": "\/tmp\/realtime\/basePersist",
Expand Down Expand Up @@ -141,7 +141,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|type|String|This should always be 'realtime'.|no|
|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)|
|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 1000000)|
|maxBytesInMemory|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size.|no (default == One-sixth of max JVM memory)|
|windowPeriod|ISO 8601 Period String|The amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server.|no (default == PT10m)|
|intermediatePersistPeriod|ISO8601 Period String|The period that determines the rate at which intermediate persists occur. These persists determine how often commits happen against the incoming realtime stream. If the realtime data loading process is interrupted at time T, it should be restarted to re-read data that arrived at T minus this period.|no (default == PT10m)|
|basePersistDirectory|String|The directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists.|no (default == java tmp dir)|
Expand Down Expand Up @@ -287,7 +288,8 @@ The following table summarizes constraints between settings in the spec file for
|segmentGranularity| Time granularity (minute, hour, day, week, month) for loading data at query time | equal to indexGranularity| more than queryGranularity|
|queryGranularity| Time granularity (minute, hour, day, week, month) for rollup | less than segmentGranularity| minute, hour, day, week, month |
|intermediatePersistPeriod| The max time (ISO8601 Period) between flushes of ingested rows from memory to disk | avoid excessive flushing | number of un-persisted rows in memory also constrained by maxRowsInMemory |
|maxRowsInMemory| The max number of ingested rows to hold in memory before a flush to disk | number of un-persisted post-aggregation rows in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod |
|maxRowsInMemory| The max number of ingested rows to hold in memory before a flush to disk. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set| number of un-persisted post-aggregation rows in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod |
|maxBytesInMemory| The number of bytes to keep in memory before a flush to disk. Normally this is computed internally and user does not need to set it. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)| number of un-persisted post-aggregation bytes in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod |

The normal, expected use cases have the following overall constraints: `intermediatePersistPeriod ≤ windowPeriod < segmentGranularity` and `queryGranularity ≤ segmentGranularity`

Expand Down
5 changes: 3 additions & 2 deletions docs/content/ingestion/tasks.md
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ The Index Task is a simpler variation of the Index Hadoop task that is designed
"tuningConfig" : {
"type" : "index",
"targetPartitionSize" : 5000000,
"maxRowsInMemory" : 75000
"maxRowsInMemory" : 1000000
}
}
}
Expand Down Expand Up @@ -137,7 +137,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|--------|-----------|-------|---------|
|type|The task type, this should always be "index".|none|yes|
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no|
|maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|75000|no|
|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no|
|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no|
|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no|
|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no|
|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
Expand Down
1 change: 0 additions & 1 deletion examples/conf-quickstart/tranquility/kafka.json
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@
},
"tuningConfig" : {
"type" : "realtime",
"maxRowsInMemory" : "100000",
"intermediatePersistPeriod" : "PT10M",
"windowPeriod" : "PT10M"
}
Expand Down
1 change: 0 additions & 1 deletion examples/conf-quickstart/tranquility/server.json
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@
},
"tuningConfig" : {
"type" : "realtime",
"maxRowsInMemory" : "100000",
"intermediatePersistPeriod" : "PT10M",
"windowPeriod" : "PT10M"
}
Expand Down
1 change: 0 additions & 1 deletion examples/conf/tranquility/kafka.json
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@
},
"tuningConfig" : {
"type" : "realtime",
"maxRowsInMemory" : "100000",
"intermediatePersistPeriod" : "PT10M",
"windowPeriod" : "PT10M"
}
Expand Down
1 change: 0 additions & 1 deletion examples/conf/tranquility/server.json
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@
},
"tuningConfig" : {
"type" : "realtime",
"maxRowsInMemory" : "100000",
"intermediatePersistPeriod" : "PT10M",
"windowPeriod" : "PT10M"
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,7 @@ public void setUp() throws Exception
null,
null,
null,
null,
false,
false,
false,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false;

private final int maxRowsInMemory;
private final long maxBytesInMemory;
private final int maxRowsPerSegment;
private final Period intermediatePersistPeriod;
private final File basePersistDirectory;
Expand All @@ -58,6 +59,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
@JsonCreator
public KafkaTuningConfig(
@JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory,
@JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory,
@JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment,
@JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod,
@JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory,
Expand All @@ -80,6 +82,9 @@ public KafkaTuningConfig(

this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory;
this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment;
// initializing this to 0, it will be lazily initialized to a value
// @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long)
this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory;
this.intermediatePersistPeriod = intermediatePersistPeriod == null
? defaults.getIntermediatePersistPeriod()
: intermediatePersistPeriod;
Expand Down Expand Up @@ -116,6 +121,7 @@ public static KafkaTuningConfig copyOf(KafkaTuningConfig config)
{
return new KafkaTuningConfig(
config.maxRowsInMemory,
config.maxBytesInMemory,
config.maxRowsPerSegment,
config.intermediatePersistPeriod,
config.basePersistDirectory,
Expand All @@ -140,6 +146,13 @@ public int getMaxRowsInMemory()
return maxRowsInMemory;
}

@Override
@JsonProperty
public long getMaxBytesInMemory()
{
return maxBytesInMemory;
}

@JsonProperty
public int getMaxRowsPerSegment()
{
Expand Down Expand Up @@ -240,6 +253,7 @@ public KafkaTuningConfig withBasePersistDirectory(File dir)
{
return new KafkaTuningConfig(
maxRowsInMemory,
maxBytesInMemory,
maxRowsPerSegment,
intermediatePersistPeriod,
dir,
Expand Down Expand Up @@ -269,6 +283,7 @@ public boolean equals(Object o)
KafkaTuningConfig that = (KafkaTuningConfig) o;
return maxRowsInMemory == that.maxRowsInMemory &&
maxRowsPerSegment == that.maxRowsPerSegment &&
maxBytesInMemory == that.maxBytesInMemory &&
maxPendingPersists == that.maxPendingPersists &&
reportParseExceptions == that.reportParseExceptions &&
handoffConditionTimeout == that.handoffConditionTimeout &&
Expand All @@ -289,6 +304,7 @@ public int hashCode()
return Objects.hash(
maxRowsInMemory,
maxRowsPerSegment,
maxBytesInMemory,
intermediatePersistPeriod,
basePersistDirectory,
maxPendingPersists,
Expand All @@ -310,6 +326,7 @@ public String toString()
return "KafkaTuningConfig{" +
"maxRowsInMemory=" + maxRowsInMemory +
", maxRowsPerSegment=" + maxRowsPerSegment +
", maxBytesInMemory=" + maxBytesInMemory +
", intermediatePersistPeriod=" + intermediatePersistPeriod +
", basePersistDirectory=" + basePersistDirectory +
", maxPendingPersists=" + maxPendingPersists +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@ public KafkaSupervisorSpec(
null,
null,
null,
null,
null
);
this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.indexing.kafka.KafkaTuningConfig;
import io.druid.segment.indexing.TuningConfigs;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.segment.IndexSpec;
import org.joda.time.Duration;
Expand All @@ -40,6 +41,7 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig

public KafkaSupervisorTuningConfig(
@JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
@JsonProperty("maxBytesInMemory") Long maxBytesInMemory,
@JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment,
@JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod,
@JsonProperty("basePersistDirectory") File basePersistDirectory,
Expand All @@ -65,6 +67,7 @@ public KafkaSupervisorTuningConfig(
{
super(
maxRowsInMemory,
maxBytesInMemory,
maxRowsPerSegment,
intermediatePersistPeriod,
basePersistDirectory,
Expand Down Expand Up @@ -131,6 +134,7 @@ public String toString()
return "KafkaSupervisorTuningConfig{" +
"maxRowsInMemory=" + getMaxRowsInMemory() +
", maxRowsPerSegment=" + getMaxRowsPerSegment() +
", maxBytesInMemory=" + TuningConfigs.getMaxBytesInMemoryOrDefault(getMaxBytesInMemory()) +
", intermediatePersistPeriod=" + getIntermediatePersistPeriod() +
", basePersistDirectory=" + getBasePersistDirectory() +
", maxPendingPersists=" + getMaxPendingPersists() +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1965,6 +1965,7 @@ private KafkaIndexTask createTask(
{
final KafkaTuningConfig tuningConfig = new KafkaTuningConfig(
1000,
null,
maxRowsPerSegment,
new Period("P1Y"),
null,
Expand Down Expand Up @@ -2007,6 +2008,7 @@ private KafkaIndexTask createTask(
{
final KafkaTuningConfig tuningConfig = new KafkaTuningConfig(
1000,
null,
maxRowsPerSegment,
new Period("P1Y"),
null,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public void testSerdeWithDefaults() throws Exception
);

Assert.assertNotNull(config.getBasePersistDirectory());
Assert.assertEquals(75000, config.getMaxRowsInMemory());
Assert.assertEquals(1000000, config.getMaxRowsInMemory());
Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment());
Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod());
Assert.assertEquals(0, config.getMaxPendingPersists());
Expand Down Expand Up @@ -103,6 +103,7 @@ public void testCopyOf()
{
KafkaTuningConfig original = new KafkaTuningConfig(
1,
null,
2,
new Period("PT3S"),
new File("/tmp/xxx"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,6 +185,7 @@ public void setupTest()

tuningConfig = new KafkaSupervisorTuningConfig(
1000,
null,
50000,
new Period("P1Y"),
new File("/test"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public void testSerdeWithDefaults() throws Exception
);

Assert.assertNotNull(config.getBasePersistDirectory());
Assert.assertEquals(75000, config.getMaxRowsInMemory());
Assert.assertEquals(1000000, config.getMaxRowsInMemory());
Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment());
Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod());
Assert.assertEquals(0, config.getMaxPendingPersists());
Expand Down
Loading

0 comments on commit 13c616b

Please sign in to comment.