Merge pull request #11437 from mikemccand/require_units

Core: time-duration and byte-sized settings now require explicit units.

On upgrade, if there are any cluster or index settings that are missing units, a warning is logged and the default unit is applied.

Closes #7616 
Closes #10888
This commit is contained in:
Michael McCandless 2015-06-04 14:03:23 -04:00
commit 6dd6ce2f8f
169 changed files with 739 additions and 265 deletions

View File

@ -240,7 +240,11 @@ def generate_index(client, version, index_name):
client.indices.create(index=index_name, body={ client.indices.create(index=index_name, body={
'settings': { 'settings': {
'number_of_shards': 1, 'number_of_shards': 1,
'number_of_replicas': 0 'number_of_replicas': 0,
# Same as ES default (60 seconds), but missing the units to make sure they are inserted on upgrade:
"gc_deletes": '60000',
# Same as ES default (5 GB), but missing the units to make sure they are inserted on upgrade:
"merge.policy.max_merged_segment": '5368709120'
}, },
'mappings': mappings 'mappings': mappings
}) })
@ -261,7 +265,11 @@ def snapshot_index(client, version, repo_dir):
# Add bogus persistent settings to make sure they can be restored # Add bogus persistent settings to make sure they can be restored
client.cluster.put_settings(body={ client.cluster.put_settings(body={
'persistent': { 'persistent': {
'cluster.routing.allocation.exclude.version_attr': version 'cluster.routing.allocation.exclude.version_attr': version,
# Same as ES default (30 seconds), but missing the units to make sure they are inserted on upgrade:
'discovery.zen.publish_timeout': '30000',
# Same as ES default (512 KB), but missing the units to make sure they are inserted on upgrade:
'indices.recovery.file_chunk_size': '524288',
} }
}) })
client.indices.put_template(name='template_' + version.lower(), order=0, body={ client.indices.put_template(name='template_' + version.lower(), order=0, body={

View File

@ -45,7 +45,7 @@
type: test type: test
id: 1 id: 1
body: { foo: bar } body: { foo: bar }
ttl: 100000 ttl: 100000ms
- do: - do:
get: get:
index: test_1 index: test_1

View File

@ -40,7 +40,7 @@
type: test type: test
id: 1 id: 1
body: { foo: bar } body: { foo: bar }
ttl: 100000 ttl: 100000ms
- do: - do:
get: get:
index: test_1 index: test_1

View File

@ -44,7 +44,7 @@
body: body:
doc: { foo: baz } doc: { foo: baz }
upsert: { foo: bar } upsert: { foo: bar }
ttl: 100000 ttl: 100000ms
- do: - do:
get: get:

View File

@ -83,7 +83,7 @@ public class ClusterHealthRequest extends MasterNodeReadRequest<ClusterHealthReq
} }
public ClusterHealthRequest timeout(String timeout) { public ClusterHealthRequest timeout(String timeout) {
return this.timeout(TimeValue.parseTimeValue(timeout, null)); return this.timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
} }
public ClusterHealthStatus waitForStatus() { public ClusterHealthStatus waitForStatus() {

View File

@ -120,7 +120,7 @@ public class DeleteIndexRequest extends MasterNodeRequest<DeleteIndexRequest> im
* to <tt>10s</tt>. * to <tt>10s</tt>.
*/ */
public DeleteIndexRequest timeout(String timeout) { public DeleteIndexRequest timeout(String timeout) {
return timeout(TimeValue.parseTimeValue(timeout, null)); return timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
} }
@Override @Override

View File

@ -317,7 +317,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
timestamp = parser.text(); timestamp = parser.text();
} else if ("_ttl".equals(currentFieldName) || "ttl".equals(currentFieldName)) { } else if ("_ttl".equals(currentFieldName) || "ttl".equals(currentFieldName)) {
if (parser.currentToken() == XContentParser.Token.VALUE_STRING) { if (parser.currentToken() == XContentParser.Token.VALUE_STRING) {
ttl = TimeValue.parseTimeValue(parser.text(), null).millis(); ttl = TimeValue.parseTimeValue(parser.text(), null, currentFieldName).millis();
} else { } else {
ttl = parser.longValue(); ttl = parser.longValue();
} }
@ -436,7 +436,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
* A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>. * A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>.
*/ */
public final BulkRequest timeout(String timeout) { public final BulkRequest timeout(String timeout) {
return timeout(TimeValue.parseTimeValue(timeout, null)); return timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
} }
public TimeValue timeout() { public TimeValue timeout() {

View File

@ -524,7 +524,7 @@ public class SearchRequest extends ActionRequest<SearchRequest> implements Indic
* If set, will enable scrolling of the search request for the specified timeout. * If set, will enable scrolling of the search request for the specified timeout.
*/ */
public SearchRequest scroll(String keepAlive) { public SearchRequest scroll(String keepAlive) {
return scroll(new Scroll(TimeValue.parseTimeValue(keepAlive, null))); return scroll(new Scroll(TimeValue.parseTimeValue(keepAlive, null, getClass().getSimpleName() + ".Scroll.keepAlive")));
} }
/** /**

View File

@ -93,7 +93,7 @@ public class SearchScrollRequest extends ActionRequest<SearchScrollRequest> {
* If set, will enable scrolling of the search request for the specified timeout. * If set, will enable scrolling of the search request for the specified timeout.
*/ */
public SearchScrollRequest scroll(String keepAlive) { public SearchScrollRequest scroll(String keepAlive) {
return scroll(new Scroll(TimeValue.parseTimeValue(keepAlive, null))); return scroll(new Scroll(TimeValue.parseTimeValue(keepAlive, null, getClass().getSimpleName() + ".keepAlive")));
} }
@Override @Override

View File

@ -51,7 +51,7 @@ public abstract class AdapterActionFuture<T, L> extends BaseFuture<T> implements
@Override @Override
public T actionGet(String timeout) { public T actionGet(String timeout) {
return actionGet(TimeValue.parseTimeValue(timeout, null)); return actionGet(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".actionGet.timeout"));
} }
@Override @Override

View File

@ -53,7 +53,7 @@ public abstract class AcknowledgedRequest<T extends MasterNodeRequest> extends M
*/ */
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public final T timeout(String timeout) { public final T timeout(String timeout) {
this.timeout = TimeValue.parseTimeValue(timeout, this.timeout); this.timeout = TimeValue.parseTimeValue(timeout, this.timeout, getClass().getSimpleName() + ".timeout");
return (T)this; return (T)this;
} }

View File

@ -56,7 +56,7 @@ public abstract class MasterNodeRequest<T extends MasterNodeRequest> extends Act
* A timeout value in case the master has not been discovered yet or disconnected. * A timeout value in case the master has not been discovered yet or disconnected.
*/ */
public final T masterNodeTimeout(String timeout) { public final T masterNodeTimeout(String timeout) {
return masterNodeTimeout(TimeValue.parseTimeValue(timeout, null)); return masterNodeTimeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".masterNodeTimeout"));
} }
public final TimeValue masterNodeTimeout() { public final TimeValue masterNodeTimeout() {

View File

@ -74,7 +74,7 @@ public abstract class BaseNodesRequest<T extends BaseNodesRequest> extends Actio
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public final T timeout(String timeout) { public final T timeout(String timeout) {
this.timeout = TimeValue.parseTimeValue(timeout, null); this.timeout = TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout");
return (T) this; return (T) this;
} }

View File

@ -121,7 +121,7 @@ public abstract class ReplicationRequest<T extends ReplicationRequest> extends A
* A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>. * A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>.
*/ */
public final T timeout(String timeout) { public final T timeout(String timeout) {
return timeout(TimeValue.parseTimeValue(timeout, null)); return timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
} }
public TimeValue timeout() { public TimeValue timeout() {

View File

@ -97,7 +97,7 @@ public abstract class InstanceShardOperationRequest<T extends InstanceShardOpera
* A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>. * A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>.
*/ */
public final T timeout(String timeout) { public final T timeout(String timeout) {
return timeout(TimeValue.parseTimeValue(timeout, null)); return timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
} }
@Override @Override

View File

@ -259,7 +259,7 @@ public class UpdateHelper extends AbstractComponent {
if (fetchedTTL instanceof Number) { if (fetchedTTL instanceof Number) {
ttl = ((Number) fetchedTTL).longValue(); ttl = ((Number) fetchedTTL).longValue();
} else { } else {
ttl = TimeValue.parseTimeValue((String) fetchedTTL, null).millis(); ttl = TimeValue.parseTimeValue((String) fetchedTTL, null, "_ttl").millis();
} }
} }
return ttl; return ttl;

View File

@ -27,24 +27,31 @@ import com.google.common.base.Predicate;
import com.google.common.collect.*; import com.google.common.collect.*;
import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cluster.*;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.*;
import org.elasticsearch.cluster.DiffableUtils.KeyedReader; import org.elasticsearch.cluster.DiffableUtils.KeyedReader;
import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
import org.elasticsearch.cluster.service.InternalClusterService;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.HppcMaps; import org.elasticsearch.common.collect.HppcMaps;
import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader; import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.*; import org.elasticsearch.common.xcontent.*;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.indices.IndexClosedException; import org.elasticsearch.indices.IndexClosedException;
import org.elasticsearch.indices.IndexMissingException; import org.elasticsearch.indices.IndexMissingException;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.warmer.IndexWarmersMetaData; import org.elasticsearch.search.warmer.IndexWarmersMetaData;
@ -55,9 +62,6 @@ import static com.google.common.collect.Lists.newArrayList;
import static com.google.common.collect.Maps.newHashMap; import static com.google.common.collect.Maps.newHashMap;
import static org.elasticsearch.common.settings.Settings.*; import static org.elasticsearch.common.settings.Settings.*;
/**
*
*/
public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData> { public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData> {
public static final MetaData PROTO = builder().build(); public static final MetaData PROTO = builder().build();
@ -257,7 +261,7 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData> {
} }
/** /**
* Returns the merges transient and persistent settings. * Returns the merged transient and persistent settings.
*/ */
public Settings settings() { public Settings settings() {
return this.settings; return this.settings;
@ -1288,6 +1292,80 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData> {
return new Builder(metaData); return new Builder(metaData);
} }
/** All known byte-sized cluster settings. */
public static final Set<String> CLUSTER_BYTES_SIZE_SETTINGS = ImmutableSet.of(
IndicesStore.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC,
RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE,
RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE,
RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC,
RecoverySettings.INDICES_RECOVERY_MAX_SIZE_PER_SEC);
/** All known time cluster settings. */
public static final Set<String> CLUSTER_TIME_SETTINGS = ImmutableSet.of(
IndicesTTLService.INDICES_TTL_INTERVAL,
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC,
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK,
RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT,
RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT,
RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT,
DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT,
DiscoverySettings.PUBLISH_TIMEOUT,
InternalClusterService.SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD);
/** As of 2.0 we require units for time and byte-sized settings. This methods adds default units to any cluster settings that don't
* specify a unit. */
public static MetaData addDefaultUnitsIfNeeded(ESLogger logger, MetaData metaData) {
Settings.Builder newPersistentSettings = null;
for(Map.Entry<String,String> ent : metaData.persistentSettings().getAsMap().entrySet()) {
String settingName = ent.getKey();
String settingValue = ent.getValue();
if (CLUSTER_BYTES_SIZE_SETTINGS.contains(settingName)) {
try {
Long.parseLong(settingValue);
} catch (NumberFormatException nfe) {
continue;
}
// It's a naked number that previously would be interpreted as default unit (bytes); now we add it:
logger.warn("byte-sized cluster setting [{}] with value [{}] is missing units; assuming default units (b) but in future versions this will be a hard error", settingName, settingValue);
if (newPersistentSettings == null) {
newPersistentSettings = Settings.builder();
newPersistentSettings.put(metaData.persistentSettings());
}
newPersistentSettings.put(settingName, settingValue + "b");
}
if (CLUSTER_TIME_SETTINGS.contains(settingName)) {
try {
Long.parseLong(settingValue);
} catch (NumberFormatException nfe) {
continue;
}
// It's a naked number that previously would be interpreted as default unit (ms); now we add it:
logger.warn("time cluster setting [{}] with value [{}] is missing units; assuming default units (ms) but in future versions this will be a hard error", settingName, settingValue);
if (newPersistentSettings == null) {
newPersistentSettings = Settings.builder();
newPersistentSettings.put(metaData.persistentSettings());
}
newPersistentSettings.put(settingName, settingValue + "ms");
}
}
if (newPersistentSettings != null) {
return new MetaData(metaData.uuid(),
metaData.version(),
metaData.transientSettings(),
newPersistentSettings.build(),
metaData.getIndices(),
metaData.getTemplates(),
metaData.getCustoms());
} else {
// No changes:
return metaData;
}
}
public static class Builder { public static class Builder {
private String uuid; private String uuid;

View File

@ -24,7 +24,11 @@ import org.elasticsearch.cluster.routing.HashFunction;
import org.elasticsearch.cluster.routing.SimpleHashFunction; import org.elasticsearch.cluster.routing.SimpleHashFunction;
import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import com.google.common.collect.ImmutableSet;
import java.util.Set;
/** /**
* This service is responsible for upgrading legacy index metadata to the current version * This service is responsible for upgrading legacy index metadata to the current version
@ -72,29 +76,30 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
/** /**
* Checks that the index can be upgraded to the current version of the master node. * Checks that the index can be upgraded to the current version of the master node.
*
* <p/> * <p/>
* If the index does need upgrade it returns the index metadata unchanged, otherwise it returns a modified index metadata. If index cannot be * If the index does not need upgrade it returns the index metadata unchanged, otherwise it returns a modified index metadata. If index
* updated the method throws an exception. * cannot be updated the method throws an exception.
*/ */
public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData) throws Exception { public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData) throws Exception {
IndexMetaData newMetaData = indexMetaData; // Throws an exception if there are too-old segments:
newMetaData = checkSupportedVersion(newMetaData); checkSupportedVersion(indexMetaData);
newMetaData = upgradeLegacyRoutingSettings(newMetaData); IndexMetaData newMetaData = upgradeLegacyRoutingSettings(indexMetaData);
newMetaData = addDefaultUnitsIfNeeded(newMetaData);
return newMetaData; return newMetaData;
} }
/** /**
* Elasticsearch 2.0 deprecated no longer supports indices with pre Lucene v4.0 segments. All indices * Elasticsearch 2.0 no longer supports indices with pre Lucene v4.0 (Elasticsearch v 0.90.0) segments. All indices
* that were created before Elasticsearch v0.90.0 should be upgraded using upgrade plugin before they can * that were created before Elasticsearch v0.90.0 should be upgraded using upgrade plugin before they can
* be open by this version of elasticsearch. * be open by this version of elasticsearch.
*/ */
private IndexMetaData checkSupportedVersion(IndexMetaData indexMetaData) throws Exception { private void checkSupportedVersion(IndexMetaData indexMetaData) throws Exception {
if (indexMetaData.getState() == IndexMetaData.State.OPEN && isSupportedVersion(indexMetaData) == false) { if (indexMetaData.getState() == IndexMetaData.State.OPEN && isSupportedVersion(indexMetaData) == false) {
throw new IllegalStateException("The index [" + indexMetaData.getIndex() + "] was created before v0.90.0 and wasn't upgraded." throw new IllegalStateException("The index [" + indexMetaData.getIndex() + "] was created before v0.90.0 and wasn't upgraded."
+ " This index should be open using a version before " + Version.CURRENT.minimumCompatibilityVersion() + " This index should be open using a version before " + Version.CURRENT.minimumCompatibilityVersion()
+ " and upgraded using the upgrade API."); + " and upgraded using the upgrade API.");
} }
return indexMetaData;
} }
/* /*
@ -139,4 +144,97 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
return indexMetaData; return indexMetaData;
} }
/** All known byte-sized settings for an index. */
public static final Set<String> INDEX_BYTES_SIZE_SETTINGS = ImmutableSet.of(
"index.buffer_size",
"index.merge.policy.floor_segment",
"index.merge.policy.max_merged_segment",
"index.merge.policy.max_merge_size",
"index.merge.policy.min_merge_size",
"index.shard.recovery.file_chunk_size",
"index.shard.recovery.translog_size",
"index.store.throttle.max_bytes_per_sec",
"index.translog.flush_threshold_size",
"index.translog.fs.buffer_size",
"index.version_map_size");
/** All known time settings for an index. */
public static final Set<String> INDEX_TIME_SETTINGS = ImmutableSet.of(
"index.gateway.wait_for_mapping_update_post_recovery",
"index.gc_deletes",
"index.indexing.slowlog.threshold.index.debug",
"index.indexing.slowlog.threshold.index.info",
"index.indexing.slowlog.threshold.index.trace",
"index.indexing.slowlog.threshold.index.warn",
"index.refresh_interval",
"index.search.slowlog.threshold.fetch.debug",
"index.search.slowlog.threshold.fetch.info",
"index.search.slowlog.threshold.fetch.trace",
"index.search.slowlog.threshold.fetch.warn",
"index.search.slowlog.threshold.query.debug",
"index.search.slowlog.threshold.query.info",
"index.search.slowlog.threshold.query.trace",
"index.search.slowlog.threshold.query.warn",
"index.shadow.wait_for_initial_commit",
"index.store.stats_refresh_interval",
"index.translog.flush_threshold_period",
"index.translog.interval",
"index.translog.sync_interval");
/**
* Elasticsearch 2.0 requires units on byte/memory and time settings; this method adds the default unit to any such settings that are
* missing units.
*/
private IndexMetaData addDefaultUnitsIfNeeded(IndexMetaData indexMetaData) {
if (indexMetaData.getCreationVersion().before(Version.V_2_0_0)) {
// TODO: can we somehow only do this *once* for a pre-2.0 index? Maybe we could stuff a "fake marker setting" here? Seems hackish...
// Created lazily if we find any settings that are missing units:
Settings settings = indexMetaData.settings();
Settings.Builder newSettings = null;
for(String byteSizeSetting : INDEX_BYTES_SIZE_SETTINGS) {
String value = settings.get(byteSizeSetting);
if (value != null) {
try {
Long.parseLong(value);
} catch (NumberFormatException nfe) {
continue;
}
// It's a naked number that previously would be interpreted as default unit (bytes); now we add it:
logger.warn("byte-sized index setting [{}] with value [{}] is missing units; assuming default units (b) but in future versions this will be a hard error", byteSizeSetting, value);
if (newSettings == null) {
newSettings = Settings.builder();
newSettings.put(settings);
}
newSettings.put(byteSizeSetting, value + "b");
}
}
for(String timeSetting : INDEX_TIME_SETTINGS) {
String value = settings.get(timeSetting);
if (value != null) {
try {
Long.parseLong(value);
} catch (NumberFormatException nfe) {
continue;
}
// It's a naked number that previously would be interpreted as default unit (ms); now we add it:
logger.warn("time index setting [{}] with value [{}] is missing units; assuming default units (ms) but in future versions this will be a hard error", timeSetting, value);
if (newSettings == null) {
newSettings = Settings.builder();
newSettings.put(settings);
}
newSettings.put(timeSetting, value + "ms");
}
}
if (newSettings != null) {
// At least one setting was changed:
return IndexMetaData.builder(indexMetaData)
.version(indexMetaData.version())
.settings(newSettings.build())
.build();
}
}
// No changes:
return indexMetaData;
}
} }

View File

@ -101,20 +101,20 @@ public class DiskThresholdDecider extends AllocationDecider {
DiskThresholdDecider.this.includeRelocations = newRelocationsSetting; DiskThresholdDecider.this.includeRelocations = newRelocationsSetting;
} }
if (newLowWatermark != null) { if (newLowWatermark != null) {
if (!validWatermarkSetting(newLowWatermark)) { if (!validWatermarkSetting(newLowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK)) {
throw new ElasticsearchParseException("Unable to parse low watermark: [" + newLowWatermark + "]"); throw new ElasticsearchParseException("Unable to parse low watermark: [" + newLowWatermark + "]");
} }
logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, newLowWatermark); logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, newLowWatermark);
DiskThresholdDecider.this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(newLowWatermark); DiskThresholdDecider.this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(newLowWatermark);
DiskThresholdDecider.this.freeBytesThresholdLow = thresholdBytesFromWatermark(newLowWatermark); DiskThresholdDecider.this.freeBytesThresholdLow = thresholdBytesFromWatermark(newLowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK);
} }
if (newHighWatermark != null) { if (newHighWatermark != null) {
if (!validWatermarkSetting(newHighWatermark)) { if (!validWatermarkSetting(newHighWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK)) {
throw new ElasticsearchParseException("Unable to parse high watermark: [" + newHighWatermark + "]"); throw new ElasticsearchParseException("Unable to parse high watermark: [" + newHighWatermark + "]");
} }
logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, newHighWatermark); logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, newHighWatermark);
DiskThresholdDecider.this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(newHighWatermark); DiskThresholdDecider.this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(newHighWatermark);
DiskThresholdDecider.this.freeBytesThresholdHigh = thresholdBytesFromWatermark(newHighWatermark); DiskThresholdDecider.this.freeBytesThresholdHigh = thresholdBytesFromWatermark(newHighWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK);
} }
if (newRerouteInterval != null) { if (newRerouteInterval != null) {
logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, newRerouteInterval); logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, newRerouteInterval);
@ -199,18 +199,18 @@ public class DiskThresholdDecider extends AllocationDecider {
String lowWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "85%"); String lowWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "85%");
String highWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "90%"); String highWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "90%");
if (!validWatermarkSetting(lowWatermark)) { if (!validWatermarkSetting(lowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK)) {
throw new ElasticsearchParseException("Unable to parse low watermark: [" + lowWatermark + "]"); throw new ElasticsearchParseException("Unable to parse low watermark: [" + lowWatermark + "]");
} }
if (!validWatermarkSetting(highWatermark)) { if (!validWatermarkSetting(highWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK)) {
throw new ElasticsearchParseException("Unable to parse high watermark: [" + highWatermark + "]"); throw new ElasticsearchParseException("Unable to parse high watermark: [" + highWatermark + "]");
} }
// Watermark is expressed in terms of used data, but we need "free" data watermark // Watermark is expressed in terms of used data, but we need "free" data watermark
this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(lowWatermark); this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(lowWatermark);
this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(highWatermark); this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(highWatermark);
this.freeBytesThresholdLow = thresholdBytesFromWatermark(lowWatermark); this.freeBytesThresholdLow = thresholdBytesFromWatermark(lowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK);
this.freeBytesThresholdHigh = thresholdBytesFromWatermark(highWatermark); this.freeBytesThresholdHigh = thresholdBytesFromWatermark(highWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK);
this.includeRelocations = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true); this.includeRelocations = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true);
this.rerouteInterval = settings.getAsTime(CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, TimeValue.timeValueSeconds(60)); this.rerouteInterval = settings.getAsTime(CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, TimeValue.timeValueSeconds(60));
@ -569,6 +569,7 @@ public class DiskThresholdDecider extends AllocationDecider {
try { try {
return RatioValue.parseRatioValue(watermark).getAsPercent(); return RatioValue.parseRatioValue(watermark).getAsPercent();
} catch (ElasticsearchParseException ex) { } catch (ElasticsearchParseException ex) {
// NOTE: this is not end-user leniency, since up above we check that it's a valid byte or percentage, and then store the two cases separately
return 100.0; return 100.0;
} }
} }
@ -577,11 +578,12 @@ public class DiskThresholdDecider extends AllocationDecider {
* Attempts to parse the watermark into a {@link ByteSizeValue}, returning * Attempts to parse the watermark into a {@link ByteSizeValue}, returning
* a ByteSizeValue of 0 bytes if the value cannot be parsed. * a ByteSizeValue of 0 bytes if the value cannot be parsed.
*/ */
public ByteSizeValue thresholdBytesFromWatermark(String watermark) { public ByteSizeValue thresholdBytesFromWatermark(String watermark, String settingName) {
try { try {
return ByteSizeValue.parseBytesSizeValue(watermark); return ByteSizeValue.parseBytesSizeValue(watermark, settingName);
} catch (ElasticsearchParseException ex) { } catch (ElasticsearchParseException ex) {
return ByteSizeValue.parseBytesSizeValue("0b"); // NOTE: this is not end-user leniency, since up above we check that it's a valid byte or percentage, and then store the two cases separately
return ByteSizeValue.parseBytesSizeValue("0b", settingName);
} }
} }
@ -589,13 +591,13 @@ public class DiskThresholdDecider extends AllocationDecider {
* Checks if a watermark string is a valid percentage or byte size value, * Checks if a watermark string is a valid percentage or byte size value,
* returning true if valid, false if invalid. * returning true if valid, false if invalid.
*/ */
public boolean validWatermarkSetting(String watermark) { public boolean validWatermarkSetting(String watermark, String settingName) {
try { try {
RatioValue.parseRatioValue(watermark); RatioValue.parseRatioValue(watermark);
return true; return true;
} catch (ElasticsearchParseException e) { } catch (ElasticsearchParseException e) {
try { try {
ByteSizeValue.parseBytesSizeValue(watermark); ByteSizeValue.parseBytesSizeValue(watermark, settingName);
return true; return true;
} catch (ElasticsearchParseException ex) { } catch (ElasticsearchParseException ex) {
return false; return false;

View File

@ -44,12 +44,14 @@ public interface Validator {
public static final Validator TIME = new Validator() { public static final Validator TIME = new Validator() {
@Override @Override
public String validate(String setting, String value) { public String validate(String setting, String value) {
if (value == null) {
throw new NullPointerException("value must not be null");
}
try { try {
if (TimeValue.parseTimeValue(value, null) == null) { // This never returns null:
return "cannot parse value [" + value + "] as time"; TimeValue.parseTimeValue(value, null, setting);
}
} catch (ElasticsearchParseException ex) { } catch (ElasticsearchParseException ex) {
return "cannot parse value [" + value + "] as time"; return ex.getMessage();
} }
return null; return null;
} }
@ -59,15 +61,16 @@ public interface Validator {
@Override @Override
public String validate(String setting, String value) { public String validate(String setting, String value) {
try { try {
TimeValue timeValue = TimeValue.parseTimeValue(value, null); if (value == null) {
if (timeValue == null) { throw new NullPointerException("value must not be null");
return "cannot parse value [" + value + "] as time";
} }
TimeValue timeValue = TimeValue.parseTimeValue(value, null, setting);
assert timeValue != null;
if (timeValue.millis() < 0) { if (timeValue.millis() < 0) {
return "cannot parse value [" + value + "] as non negative time"; return "cannot parse value [" + value + "] as non negative time";
} }
} catch (ElasticsearchParseException ex) { } catch (ElasticsearchParseException ex) {
return "cannot parse value [" + value + "] as time"; return ex.getMessage();
} }
return null; return null;
} }
@ -197,7 +200,7 @@ public interface Validator {
@Override @Override
public String validate(String setting, String value) { public String validate(String setting, String value) {
try { try {
parseBytesSizeValue(value); parseBytesSizeValue(value, setting);
} catch (ElasticsearchParseException ex) { } catch (ElasticsearchParseException ex) {
return ex.getMessage(); return ex.getMessage();
} }
@ -247,7 +250,7 @@ public interface Validator {
@Override @Override
public String validate(String setting, String value) { public String validate(String setting, String value) {
try { try {
parseBytesSizeValueOrHeapRatio(value); parseBytesSizeValueOrHeapRatio(value, setting);
} catch (ElasticsearchParseException ex) { } catch (ElasticsearchParseException ex) {
return ex.getMessage(); return ex.getMessage();
} }

View File

@ -25,11 +25,11 @@ package org.elasticsearch.common;
public class Booleans { public class Booleans {
/** /**
* Returns <code>true</code> iff the sequence is neither of the following: * Returns <code>false</code> if text is in <tt>false</tt>, <tt>0</tt>, <tt>off</tt>, <tt>no</tt>; else, true
* <tt>false</tt>, <tt>0</tt>, <tt>off</tt>, <tt>no</tt>,
* otherwise <code>false</code>
*/ */
public static boolean parseBoolean(char[] text, int offset, int length, boolean defaultValue) { public static boolean parseBoolean(char[] text, int offset, int length, boolean defaultValue) {
// TODO: the leniency here is very dangerous: a simple typo will be misinterpreted and the user won't know.
// We should remove it and cutover to https://github.com/rmuir/booleanparser
if (text == null || length == 0) { if (text == null || length == 0) {
return defaultValue; return defaultValue;
} }
@ -84,7 +84,7 @@ public class Booleans {
* @return true/false * @return true/false
* throws exception if string cannot be parsed to boolean * throws exception if string cannot be parsed to boolean
*/ */
public static Boolean parseBooleanExact(String value){ public static Boolean parseBooleanExact(String value) {
boolean isFalse = isExplicitFalse(value); boolean isFalse = isExplicitFalse(value);
if (isFalse) { if (isFalse) {

View File

@ -56,12 +56,25 @@ import static org.elasticsearch.common.unit.SizeValue.parseSizeValue;
import static org.elasticsearch.common.unit.TimeValue.parseTimeValue; import static org.elasticsearch.common.unit.TimeValue.parseTimeValue;
/** /**
* An immutable {@code Settings} implementation. * An immutable settings implementation.
*/ */
public final class Settings implements ToXContent { public final class Settings implements ToXContent {
public static final Settings EMPTY = new Builder().build(); public static final Settings EMPTY = new Builder().build();
private final static Pattern ARRAY_PATTERN = Pattern.compile("(.*)\\.\\d+$"); private static final Pattern ARRAY_PATTERN = Pattern.compile("(.*)\\.\\d+$");
/** Name of the setting to use to disable required units for byte size, time settings. */
public static final String SETTINGS_REQUIRE_UNITS = "settings_require_units";
private static boolean settingsRequireUnits = true;
public static void setSettingsRequireUnits(boolean v) {
settingsRequireUnits = v;
}
public static boolean getSettingsRequireUnits() {
return settingsRequireUnits;
}
private ImmutableMap<String, String> settings; private ImmutableMap<String, String> settings;
private final ImmutableMap<String, String> forcedUnderscoreSettings; private final ImmutableMap<String, String> forcedUnderscoreSettings;
@ -417,7 +430,7 @@ public final class Settings implements ToXContent {
* returns the default value provided. * returns the default value provided.
*/ */
public TimeValue getAsTime(String setting, TimeValue defaultValue) { public TimeValue getAsTime(String setting, TimeValue defaultValue) {
return parseTimeValue(get(setting), defaultValue); return parseTimeValue(get(setting), defaultValue, setting);
} }
/** /**
@ -425,7 +438,14 @@ public final class Settings implements ToXContent {
* returns the default value provided. * returns the default value provided.
*/ */
public TimeValue getAsTime(String[] settings, TimeValue defaultValue) { public TimeValue getAsTime(String[] settings, TimeValue defaultValue) {
return parseTimeValue(get(settings), defaultValue); // NOTE: duplicated from get(String[]) so we can pass which setting name was actually used to parseTimeValue:
for (String setting : settings) {
String retVal = get(setting);
if (retVal != null) {
parseTimeValue(get(settings), defaultValue, setting);
}
}
return defaultValue;
} }
/** /**
@ -433,7 +453,7 @@ public final class Settings implements ToXContent {
* returns the default value provided. * returns the default value provided.
*/ */
public ByteSizeValue getAsBytesSize(String setting, ByteSizeValue defaultValue) throws SettingsException { public ByteSizeValue getAsBytesSize(String setting, ByteSizeValue defaultValue) throws SettingsException {
return parseBytesSizeValue(get(setting), defaultValue); return parseBytesSizeValue(get(setting), defaultValue, setting);
} }
/** /**
@ -441,7 +461,14 @@ public final class Settings implements ToXContent {
* returns the default value provided. * returns the default value provided.
*/ */
public ByteSizeValue getAsBytesSize(String[] settings, ByteSizeValue defaultValue) throws SettingsException { public ByteSizeValue getAsBytesSize(String[] settings, ByteSizeValue defaultValue) throws SettingsException {
return parseBytesSizeValue(get(settings), defaultValue); // NOTE: duplicated from get(String[]) so we can pass which setting name was actually used to parseBytesSizeValue
for (String setting : settings) {
String retVal = get(setting);
if (retVal != null) {
parseBytesSizeValue(get(settings), defaultValue, setting);
}
}
return defaultValue;
} }
/** /**
@ -450,7 +477,7 @@ public final class Settings implements ToXContent {
* (eg. 12%). If it does not exists, parses the default value provided. * (eg. 12%). If it does not exists, parses the default value provided.
*/ */
public ByteSizeValue getAsMemory(String setting, String defaultValue) throws SettingsException { public ByteSizeValue getAsMemory(String setting, String defaultValue) throws SettingsException {
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(get(setting, defaultValue)); return MemorySizeValue.parseBytesSizeValueOrHeapRatio(get(setting, defaultValue), setting);
} }
/** /**
@ -459,7 +486,14 @@ public final class Settings implements ToXContent {
* (eg. 12%). If it does not exists, parses the default value provided. * (eg. 12%). If it does not exists, parses the default value provided.
*/ */
public ByteSizeValue getAsMemory(String[] settings, String defaultValue) throws SettingsException { public ByteSizeValue getAsMemory(String[] settings, String defaultValue) throws SettingsException {
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(get(settings, defaultValue)); // NOTE: duplicated from get(String[]) so we can pass which setting name was actually used to parseBytesSizeValueOrHeapRatio
for (String setting : settings) {
String retVal = get(setting);
if (retVal != null) {
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(retVal, setting);
}
}
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(defaultValue, settings[0]);
} }
/** /**
@ -953,7 +987,7 @@ public final class Settings implements ToXContent {
* @return The builder * @return The builder
*/ */
public Builder put(String setting, long value, TimeUnit timeUnit) { public Builder put(String setting, long value, TimeUnit timeUnit) {
put(setting, timeUnit.toMillis(value)); put(setting, timeUnit.toMillis(value) + "ms");
return this; return this;
} }
@ -965,7 +999,7 @@ public final class Settings implements ToXContent {
* @return The builder * @return The builder
*/ */
public Builder put(String setting, long value, ByteSizeUnit sizeUnit) { public Builder put(String setting, long value, ByteSizeUnit sizeUnit) {
put(setting, sizeUnit.toBytes(value)); put(setting, sizeUnit.toBytes(value) + "b");
return this; return this;
} }

View File

@ -20,18 +20,18 @@
package org.elasticsearch.common.unit; package org.elasticsearch.common.unit;
import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable; import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.Locale; import java.util.Locale;
import java.util.Objects;
/**
*
*/
public class ByteSizeValue implements Serializable, Streamable { public class ByteSizeValue implements Serializable, Streamable {
private long size; private long size;
@ -171,41 +171,55 @@ public class ByteSizeValue implements Serializable, Streamable {
return Strings.format1Decimals(value, suffix); return Strings.format1Decimals(value, suffix);
} }
public static ByteSizeValue parseBytesSizeValue(String sValue) throws ElasticsearchParseException { public static ByteSizeValue parseBytesSizeValue(String sValue, String settingName) throws ElasticsearchParseException {
return parseBytesSizeValue(sValue, null); return parseBytesSizeValue(sValue, null, settingName);
} }
public static ByteSizeValue parseBytesSizeValue(String sValue, ByteSizeValue defaultValue) throws ElasticsearchParseException { public static ByteSizeValue parseBytesSizeValue(String sValue, ByteSizeValue defaultValue, String settingName) throws ElasticsearchParseException {
settingName = Objects.requireNonNull(settingName);
assert settingName.startsWith("index.") == false || MetaDataIndexUpgradeService.INDEX_BYTES_SIZE_SETTINGS.contains(settingName);
if (sValue == null) { if (sValue == null) {
return defaultValue; return defaultValue;
} }
long bytes; long bytes;
try { try {
String lastTwoChars = sValue.substring(sValue.length() - Math.min(2, sValue.length())).toLowerCase(Locale.ROOT); String lowerSValue = sValue.toLowerCase(Locale.ROOT).trim();
if (lastTwoChars.endsWith("k")) { if (lowerSValue.endsWith("k")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C1); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C1);
} else if (lastTwoChars.endsWith("kb")) { } else if (lowerSValue.endsWith("kb")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C1); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C1);
} else if (lastTwoChars.endsWith("m")) { } else if (lowerSValue.endsWith("m")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C2); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C2);
} else if (lastTwoChars.endsWith("mb")) { } else if (lowerSValue.endsWith("mb")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C2); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C2);
} else if (lastTwoChars.endsWith("g")) { } else if (lowerSValue.endsWith("g")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C3); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C3);
} else if (lastTwoChars.endsWith("gb")) { } else if (lowerSValue.endsWith("gb")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C3); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C3);
} else if (lastTwoChars.endsWith("t")) { } else if (lowerSValue.endsWith("t")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C4); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C4);
} else if (lastTwoChars.endsWith("tb")) { } else if (lowerSValue.endsWith("tb")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C4); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C4);
} else if (lastTwoChars.endsWith("p")) { } else if (lowerSValue.endsWith("p")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C5); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C5);
} else if (lastTwoChars.endsWith("pb")) { } else if (lowerSValue.endsWith("pb")) {
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C5); bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C5);
} else if (lastTwoChars.endsWith("b")) { } else if (lowerSValue.endsWith("b")) {
bytes = Long.parseLong(sValue.substring(0, sValue.length() - 1)); bytes = Long.parseLong(lowerSValue.substring(0, lowerSValue.length() - 1).trim());
} else if (lowerSValue.equals("-1")) {
// Allow this special value to be unit-less:
bytes = -1;
} else if (lowerSValue.equals("0")) {
// Allow this special value to be unit-less:
bytes = 0;
} else { } else {
bytes = Long.parseLong(sValue); // Missing units:
if (Settings.getSettingsRequireUnits()) {
throw new ElasticsearchParseException("Failed to parse setting [" + settingName + "] with value [" + sValue + "] as a size in bytes: unit is missing or unrecognized") ;
} else {
// Leniency default to bytes:
bytes = Long.parseLong(sValue);
}
} }
} catch (NumberFormatException e) { } catch (NumberFormatException e) {
throw new ElasticsearchParseException("Failed to parse [" + sValue + "]", e); throw new ElasticsearchParseException("Failed to parse [" + sValue + "]", e);

View File

@ -152,7 +152,7 @@ public final class Fuzziness implements ToXContent {
if (this == AUTO) { if (this == AUTO) {
return TimeValue.timeValueMillis(1); return TimeValue.timeValueMillis(1);
} else { } else {
return TimeValue.parseTimeValue(fuzziness.toString(), null); return TimeValue.parseTimeValue(fuzziness.toString(), null, "fuzziness");
} }
} }

View File

@ -19,6 +19,8 @@
package org.elasticsearch.common.unit; package org.elasticsearch.common.unit;
import java.util.Objects;
import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.monitor.jvm.JvmInfo;
@ -31,7 +33,8 @@ public enum MemorySizeValue {
/** Parse the provided string as a memory size. This method either accepts absolute values such as /** Parse the provided string as a memory size. This method either accepts absolute values such as
* <tt>42</tt> (default assumed unit is byte) or <tt>2mb</tt>, or percentages of the heap size: if * <tt>42</tt> (default assumed unit is byte) or <tt>2mb</tt>, or percentages of the heap size: if
* the heap is 1G, <tt>10%</tt> will be parsed as <tt>100mb</tt>. */ * the heap is 1G, <tt>10%</tt> will be parsed as <tt>100mb</tt>. */
public static ByteSizeValue parseBytesSizeValueOrHeapRatio(String sValue) { public static ByteSizeValue parseBytesSizeValueOrHeapRatio(String sValue, String settingName) {
settingName = Objects.requireNonNull(settingName);
if (sValue != null && sValue.endsWith("%")) { if (sValue != null && sValue.endsWith("%")) {
final String percentAsString = sValue.substring(0, sValue.length() - 1); final String percentAsString = sValue.substring(0, sValue.length() - 1);
try { try {
@ -44,7 +47,7 @@ public enum MemorySizeValue {
throw new ElasticsearchParseException("Failed to parse [" + percentAsString + "] as a double", e); throw new ElasticsearchParseException("Failed to parse [" + percentAsString + "] as a double", e);
} }
} else { } else {
return parseBytesSizeValue(sValue); return parseBytesSizeValue(sValue, settingName);
} }
} }
} }

View File

@ -20,10 +20,12 @@
package org.elasticsearch.common.unit; package org.elasticsearch.common.unit;
import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable; import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings;
import org.joda.time.Period; import org.joda.time.Period;
import org.joda.time.PeriodType; import org.joda.time.PeriodType;
import org.joda.time.format.PeriodFormat; import org.joda.time.format.PeriodFormat;
@ -31,11 +33,10 @@ import org.joda.time.format.PeriodFormatter;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.Locale;
import java.util.Objects;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
/**
*
*/
public class TimeValue implements Serializable, Streamable { public class TimeValue implements Serializable, Streamable {
/** How many nano-seconds in one milli-second */ /** How many nano-seconds in one milli-second */
@ -228,28 +229,41 @@ public class TimeValue implements Serializable, Streamable {
return Strings.format1Decimals(value, suffix); return Strings.format1Decimals(value, suffix);
} }
public static TimeValue parseTimeValue(String sValue, TimeValue defaultValue) { public static TimeValue parseTimeValue(String sValue, TimeValue defaultValue, String settingName) {
settingName = Objects.requireNonNull(settingName);
assert settingName.startsWith("index.") == false || MetaDataIndexUpgradeService.INDEX_TIME_SETTINGS.contains(settingName);
if (sValue == null) { if (sValue == null) {
return defaultValue; return defaultValue;
} }
try { try {
long millis; long millis;
if (sValue.endsWith("S")) { String lowerSValue = sValue.toLowerCase(Locale.ROOT).trim();
millis = Long.parseLong(sValue.substring(0, sValue.length() - 1)); if (lowerSValue.endsWith("ms")) {
} else if (sValue.endsWith("ms")) { millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)));
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2))); } else if (lowerSValue.endsWith("s")) {
} else if (sValue.endsWith("s")) { millis = (long) Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 1000;
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 1000); } else if (lowerSValue.endsWith("m")) {
} else if (sValue.endsWith("m")) { millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 60 * 1000);
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 60 * 1000); } else if (lowerSValue.endsWith("h")) {
} else if (sValue.endsWith("H") || sValue.endsWith("h")) { millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 60 * 60 * 1000);
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 60 * 60 * 1000); } else if (lowerSValue.endsWith("d")) {
} else if (sValue.endsWith("d")) { millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 24 * 60 * 60 * 1000);
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 24 * 60 * 60 * 1000); } else if (lowerSValue.endsWith("w")) {
} else if (sValue.endsWith("w")) { millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 7 * 24 * 60 * 60 * 1000);
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 7 * 24 * 60 * 60 * 1000); } else if (lowerSValue.equals("-1")) {
// Allow this special value to be unit-less:
millis = -1;
} else if (lowerSValue.equals("0")) {
// Allow this special value to be unit-less:
millis = 0;
} else { } else {
millis = Long.parseLong(sValue); if (Settings.getSettingsRequireUnits()) {
// Missing units:
throw new ElasticsearchParseException("Failed to parse setting [" + settingName + "] with value [" + sValue + "] as a time value: unit is missing or unrecognized");
} else {
// Leniency default to msec for bwc:
millis = Long.parseLong(sValue);
}
} }
return new TimeValue(millis, TimeUnit.MILLISECONDS); return new TimeValue(millis, TimeUnit.MILLISECONDS);
} catch (NumberFormatException e) { } catch (NumberFormatException e) {

View File

@ -377,7 +377,7 @@ public class XContentMapValues {
if (node instanceof Number) { if (node instanceof Number) {
return TimeValue.timeValueMillis(((Number) node).longValue()); return TimeValue.timeValueMillis(((Number) node).longValue());
} }
return TimeValue.parseTimeValue(node.toString(), null); return TimeValue.parseTimeValue(node.toString(), null, XContentMapValues.class.getSimpleName() + ".nodeTimeValue");
} }
public static Map<String, Object> nodeMapValue(Object node, String desc) { public static Map<String, Object> nodeMapValue(Object node, String desc) {

View File

@ -214,7 +214,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
/** /**
* Elasticsearch 2.0 removed several deprecated features and as well as support for Lucene 3.x. This method calls * Elasticsearch 2.0 removed several deprecated features and as well as support for Lucene 3.x. This method calls
* {@link MetaDataIndexUpgradeService} to makes sure that indices are compatible with the current version. The * {@link MetaDataIndexUpgradeService} to makes sure that indices are compatible with the current version. The
* MetaDataIndexUpgradeService might also update updates obsolete settings if needed. When this happens we rewrite * MetaDataIndexUpgradeService might also update obsolete settings if needed. When this happens we rewrite
* index metadata with new settings. * index metadata with new settings.
*/ */
private void pre20Upgrade() throws Exception { private void pre20Upgrade() throws Exception {

View File

@ -116,7 +116,14 @@ public class MetaStateService extends AbstractComponent {
* Loads the global state, *without* index state, see {@link #loadFullState()} for that. * Loads the global state, *without* index state, see {@link #loadFullState()} for that.
*/ */
MetaData loadGlobalState() throws IOException { MetaData loadGlobalState() throws IOException {
return globalStateFormat.loadLatestState(logger, nodeEnv.nodeDataPaths()); MetaData globalState = globalStateFormat.loadLatestState(logger, nodeEnv.nodeDataPaths());
// ES 2.0 now requires units for all time and byte-sized settings, so we add the default unit if it's missing
// TODO: can we somehow only do this for pre-2.0 cluster state?
if (globalState != null) {
return MetaData.addDefaultUnitsIfNeeded(logger, globalState);
} else {
return null;
}
} }
/** /**

View File

@ -128,8 +128,8 @@ public final class EngineConfig {
public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS); public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60); public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60);
public static final ByteSizeValue DEFAUTL_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB); public static final ByteSizeValue DEFAULT_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB);
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb"); public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb", "INACTIVE_SHARD_INDEXING_BUFFER");
public static final String DEFAULT_VERSION_MAP_SIZE = "25%"; public static final String DEFAULT_VERSION_MAP_SIZE = "25%";
@ -163,7 +163,7 @@ public final class EngineConfig {
this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush); this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush);
this.indexConcurrency = indexSettings.getAsInt(EngineConfig.INDEX_CONCURRENCY_SETTING, Math.max(IndexWriterConfig.DEFAULT_MAX_THREAD_STATES, (int) (EsExecutors.boundedNumberOfProcessors(indexSettings) * 0.65))); this.indexConcurrency = indexSettings.getAsInt(EngineConfig.INDEX_CONCURRENCY_SETTING, Math.max(IndexWriterConfig.DEFAULT_MAX_THREAD_STATES, (int) (EsExecutors.boundedNumberOfProcessors(indexSettings) * 0.65)));
codecName = indexSettings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME); codecName = indexSettings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME);
indexingBufferSize = indexSettings.getAsBytesSize(INDEX_BUFFER_SIZE_SETTING, DEFAUTL_INDEX_BUFFER_SIZE); indexingBufferSize = indexSettings.getAsBytesSize(INDEX_BUFFER_SIZE_SETTING, DEFAULT_INDEX_BUFFER_SIZE);
gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis(); gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis();
versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE); versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
updateVersionMapSize(); updateVersionMapSize();
@ -180,7 +180,7 @@ public final class EngineConfig {
double percent = Double.parseDouble(versionMapSizeSetting.substring(0, versionMapSizeSetting.length() - 1)); double percent = Double.parseDouble(versionMapSizeSetting.substring(0, versionMapSizeSetting.length() - 1));
versionMapSize = new ByteSizeValue((long) (((double) indexingBufferSize.bytes() * (percent / 100)))); versionMapSize = new ByteSizeValue((long) (((double) indexingBufferSize.bytes() * (percent / 100))));
} else { } else {
versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting); versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting, INDEX_VERSION_MAP_SIZE);
} }
} }

View File

@ -170,7 +170,7 @@ public class SourceFieldMapper extends AbstractFieldMapper implements RootMapper
builder.compressThreshold(((Number) fieldNode).longValue()); builder.compressThreshold(((Number) fieldNode).longValue());
builder.compress(true); builder.compress(true);
} else { } else {
builder.compressThreshold(ByteSizeValue.parseBytesSizeValue(fieldNode.toString()).bytes()); builder.compressThreshold(ByteSizeValue.parseBytesSizeValue(fieldNode.toString(), "compress_threshold").bytes());
builder.compress(true); builder.compress(true);
} }
} }

View File

@ -207,7 +207,7 @@ public class TTLFieldMapper extends LongFieldMapper implements RootMapper {
if (context.sourceToParse().ttl() < 0) { // no ttl has been provided externally if (context.sourceToParse().ttl() < 0) { // no ttl has been provided externally
long ttl; long ttl;
if (context.parser().currentToken() == XContentParser.Token.VALUE_STRING) { if (context.parser().currentToken() == XContentParser.Token.VALUE_STRING) {
ttl = TimeValue.parseTimeValue(context.parser().text(), null).millis(); ttl = TimeValue.parseTimeValue(context.parser().text(), null, "ttl").millis();
} else { } else {
ttl = context.parser().longValue(coerce.value()); ttl = context.parser().longValue(coerce.value());
} }

View File

@ -269,9 +269,9 @@ public abstract class DecayFunctionParser implements ScoreFunctionParser {
if (scaleString == null) { if (scaleString == null) {
throw new ElasticsearchParseException(DecayFunctionBuilder.SCALE + " must be set for date fields."); throw new ElasticsearchParseException(DecayFunctionBuilder.SCALE + " must be set for date fields.");
} }
TimeValue val = TimeValue.parseTimeValue(scaleString, TimeValue.timeValueHours(24)); TimeValue val = TimeValue.parseTimeValue(scaleString, TimeValue.timeValueHours(24), getClass().getSimpleName() + ".scale");
double scale = val.getMillis(); double scale = val.getMillis();
val = TimeValue.parseTimeValue(offsetString, TimeValue.timeValueHours(24)); val = TimeValue.parseTimeValue(offsetString, TimeValue.timeValueHours(24), getClass().getSimpleName() + ".offset");
double offset = val.getMillis(); double offset = val.getMillis();
IndexNumericFieldData numericFieldData = parseContext.getForField(dateFieldMapper); IndexNumericFieldData numericFieldData = parseContext.getForField(dateFieldMapper);
return new NumericFieldDataScoreFunction(origin, scale, decay, offset, getDecayFunction(), numericFieldData, mode); return new NumericFieldDataScoreFunction(origin, scale, decay, offset, getDecayFunction(), numericFieldData, mode);

View File

@ -42,7 +42,7 @@ public final class TranslogConfig {
public static final String INDEX_TRANSLOG_FS_TYPE = "index.translog.fs.type"; public static final String INDEX_TRANSLOG_FS_TYPE = "index.translog.fs.type";
public static final String INDEX_TRANSLOG_BUFFER_SIZE = "index.translog.fs.buffer_size"; public static final String INDEX_TRANSLOG_BUFFER_SIZE = "index.translog.fs.buffer_size";
public static final String INDEX_TRANSLOG_SYNC_INTERVAL = "index.translog.sync_interval"; public static final String INDEX_TRANSLOG_SYNC_INTERVAL = "index.translog.sync_interval";
public static final ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb"); public static final ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb", "INACTIVE_SHARD_TRANSLOG_BUFFER");
private final TimeValue syncInterval; private final TimeValue syncInterval;
private final BigArrays bigArrays; private final BigArrays bigArrays;
@ -73,7 +73,7 @@ public final class TranslogConfig {
this.threadPool = threadPool; this.threadPool = threadPool;
this.bigArrays = bigArrays; this.bigArrays = bigArrays;
this.type = TranslogWriter.Type.fromString(indexSettings.get(INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.BUFFERED.name())); this.type = TranslogWriter.Type.fromString(indexSettings.get(INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.BUFFERED.name()));
this.bufferSize = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, ByteSizeValue.parseBytesSizeValue("64k")).bytes(); // Not really interesting, updated by IndexingMemoryController... this.bufferSize = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, ByteSizeValue.parseBytesSizeValue("64k", INDEX_TRANSLOG_BUFFER_SIZE)).bytes(); // Not really interesting, updated by IndexingMemoryController...
syncInterval = indexSettings.getAsTime(INDEX_TRANSLOG_SYNC_INTERVAL, TimeValue.timeValueSeconds(5)); syncInterval = indexSettings.getAsTime(INDEX_TRANSLOG_SYNC_INTERVAL, TimeValue.timeValueSeconds(5));
if (syncInterval.millis() > 0 && threadPool != null) { if (syncInterval.millis() > 0 && threadPool != null) {

View File

@ -65,7 +65,7 @@ public class IndicesFilterCache extends AbstractComponent implements QueryCache,
public IndicesFilterCache(Settings settings) { public IndicesFilterCache(Settings settings) {
super(settings); super(settings);
final String sizeString = settings.get(INDICES_CACHE_QUERY_SIZE, "10%"); final String sizeString = settings.get(INDICES_CACHE_QUERY_SIZE, "10%");
final ByteSizeValue size = MemorySizeValue.parseBytesSizeValueOrHeapRatio(sizeString); final ByteSizeValue size = MemorySizeValue.parseBytesSizeValueOrHeapRatio(sizeString, INDICES_CACHE_QUERY_SIZE);
final int count = settings.getAsInt(INDICES_CACHE_QUERY_COUNT, 100000); final int count = settings.getAsInt(INDICES_CACHE_QUERY_COUNT, 100000);
logger.debug("using [node] weighted filter cache with size [{}], actual_size [{}], max filter count [{}]", logger.debug("using [node] weighted filter cache with size [{}], actual_size [{}], max filter count [{}]",
sizeString, size, count); sizeString, size, count);

View File

@ -128,7 +128,7 @@ public class IndicesQueryCache extends AbstractComponent implements RemovalListe
} }
private void buildCache() { private void buildCache() {
long sizeInBytes = MemorySizeValue.parseBytesSizeValueOrHeapRatio(size).bytes(); long sizeInBytes = MemorySizeValue.parseBytesSizeValueOrHeapRatio(size, INDICES_CACHE_QUERY_SIZE).bytes();
CacheBuilder<Key, Value> cacheBuilder = CacheBuilder.newBuilder() CacheBuilder<Key, Value> cacheBuilder = CacheBuilder.newBuilder()
.maximumWeight(sizeInBytes).weigher(new QueryCacheWeigher()).removalListener(this); .maximumWeight(sizeInBytes).weigher(new QueryCacheWeigher()).removalListener(this);

View File

@ -73,7 +73,7 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
public static final String INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT = "indices.recovery.internal_action_long_timeout"; public static final String INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT = "indices.recovery.internal_action_long_timeout";
public static final long SMALL_FILE_CUTOFF_BYTES = ByteSizeValue.parseBytesSizeValue("5mb").bytes(); public static final long SMALL_FILE_CUTOFF_BYTES = ByteSizeValue.parseBytesSizeValue("5mb", "SMALL_FILE_CUTOFF_BYTES").bytes();
/** /**
* Use {@link #INDICES_RECOVERY_MAX_BYTES_PER_SEC} instead * Use {@link #INDICES_RECOVERY_MAX_BYTES_PER_SEC} instead

View File

@ -22,6 +22,7 @@ package org.elasticsearch.node.internal;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.UnmodifiableIterator; import com.google.common.collect.UnmodifiableIterator;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Names; import org.elasticsearch.common.Names;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.cli.Terminal; import org.elasticsearch.common.cli.Terminal;
@ -153,6 +154,10 @@ public class InternalSettingsPreparer {
settingsBuilder.put(ClusterName.SETTING, ClusterName.DEFAULT.value()); settingsBuilder.put(ClusterName.SETTING, ClusterName.DEFAULT.value());
} }
String v = settingsBuilder.get(Settings.SETTINGS_REQUIRE_UNITS);
if (v != null) {
Settings.setSettingsRequireUnits(Booleans.parseBoolean(v, true));
}
Settings v1 = replacePromptPlaceholders(settingsBuilder.build(), terminal); Settings v1 = replacePromptPlaceholders(settingsBuilder.build(), terminal);
environment = new Environment(v1); environment = new Environment(v1);

View File

@ -409,7 +409,7 @@ public class PluginManager {
case "timeout": case "timeout":
case "-timeout": case "-timeout":
String timeoutValue = getCommandValue(args, ++c, "--timeout"); String timeoutValue = getCommandValue(args, ++c, "--timeout");
timeout = TimeValue.parseTimeValue(timeoutValue, DEFAULT_TIMEOUT); timeout = TimeValue.parseTimeValue(timeoutValue, DEFAULT_TIMEOUT, command);
break; break;
case "-l": case "-l":
case "--list": case "--list":

View File

@ -140,11 +140,11 @@ public abstract class RestRequest extends ContextAndHeaderHolder implements ToXC
} }
public TimeValue paramAsTime(String key, TimeValue defaultValue) { public TimeValue paramAsTime(String key, TimeValue defaultValue) {
return parseTimeValue(param(key), defaultValue); return parseTimeValue(param(key), defaultValue, key);
} }
public ByteSizeValue paramAsSize(String key, ByteSizeValue defaultValue) { public ByteSizeValue paramAsSize(String key, ByteSizeValue defaultValue) {
return parseBytesSizeValue(param(key), defaultValue); return parseBytesSizeValue(param(key), defaultValue, key);
} }
public String[] paramAsStringArray(String key, String[] defaultValue) { public String[] paramAsStringArray(String key, String[] defaultValue) {

View File

@ -56,7 +56,7 @@ public class RestNodesHotThreadsAction extends BaseRestHandler {
nodesHotThreadsRequest.threads(request.paramAsInt("threads", nodesHotThreadsRequest.threads())); nodesHotThreadsRequest.threads(request.paramAsInt("threads", nodesHotThreadsRequest.threads()));
nodesHotThreadsRequest.ignoreIdleThreads(request.paramAsBoolean("ignore_idle_threads", nodesHotThreadsRequest.ignoreIdleThreads())); nodesHotThreadsRequest.ignoreIdleThreads(request.paramAsBoolean("ignore_idle_threads", nodesHotThreadsRequest.ignoreIdleThreads()));
nodesHotThreadsRequest.type(request.param("type", nodesHotThreadsRequest.type())); nodesHotThreadsRequest.type(request.param("type", nodesHotThreadsRequest.type()));
nodesHotThreadsRequest.interval(TimeValue.parseTimeValue(request.param("interval"), nodesHotThreadsRequest.interval())); nodesHotThreadsRequest.interval(TimeValue.parseTimeValue(request.param("interval"), nodesHotThreadsRequest.interval(), "interval"));
nodesHotThreadsRequest.snapshots(request.paramAsInt("snapshots", nodesHotThreadsRequest.snapshots())); nodesHotThreadsRequest.snapshots(request.paramAsInt("snapshots", nodesHotThreadsRequest.snapshots()));
client.admin().cluster().nodesHotThreads(nodesHotThreadsRequest, new RestResponseListener<NodesHotThreadsResponse>(channel) { client.admin().cluster().nodesHotThreads(nodesHotThreadsRequest, new RestResponseListener<NodesHotThreadsResponse>(channel) {
@Override @Override

View File

@ -113,7 +113,7 @@ public class RestSearchAction extends BaseRestHandler {
String scroll = request.param("scroll"); String scroll = request.param("scroll");
if (scroll != null) { if (scroll != null) {
searchRequest.scroll(new Scroll(parseTimeValue(scroll, null))); searchRequest.scroll(new Scroll(parseTimeValue(scroll, null, "scroll")));
} }
searchRequest.types(Strings.splitStringByCommaToArray(request.param("type"))); searchRequest.types(Strings.splitStringByCommaToArray(request.param("type")));

View File

@ -63,7 +63,7 @@ public class RestSearchScrollAction extends BaseRestHandler {
searchScrollRequest.scrollId(scrollId); searchScrollRequest.scrollId(scrollId);
String scroll = request.param("scroll"); String scroll = request.param("scroll");
if (scroll != null) { if (scroll != null) {
searchScrollRequest.scroll(new Scroll(parseTimeValue(scroll, null))); searchScrollRequest.scroll(new Scroll(parseTimeValue(scroll, null, "scroll")));
} }
if (RestActions.hasBodyContent(request)) { if (RestActions.hasBodyContent(request)) {
@ -94,7 +94,7 @@ public class RestSearchScrollAction extends BaseRestHandler {
} else if ("scroll_id".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) { } else if ("scroll_id".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) {
searchScrollRequest.scrollId(parser.text()); searchScrollRequest.scrollId(parser.text());
} else if ("scroll".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) { } else if ("scroll".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) {
searchScrollRequest.scroll(new Scroll(TimeValue.parseTimeValue(parser.text(), null))); searchScrollRequest.scroll(new Scroll(TimeValue.parseTimeValue(parser.text(), null, "scroll")));
} else { } else {
throw new IllegalArgumentException("Unknown parameter [" + currentFieldName + "] in request body or parameter is of the wrong type[" + token + "] "); throw new IllegalArgumentException("Unknown parameter [" + currentFieldName + "] in request body or parameter is of the wrong type[" + token + "] ");
} }

View File

@ -63,4 +63,4 @@ public class DateHistogramInterval {
public String toString() { public String toString() {
return expression; return expression;
} }
} }

View File

@ -189,7 +189,7 @@ public class DateHistogramParser implements Aggregator.Parser {
tzRoundingBuilder = TimeZoneRounding.builder(dateTimeUnit); tzRoundingBuilder = TimeZoneRounding.builder(dateTimeUnit);
} else { } else {
// the interval is a time value? // the interval is a time value?
tzRoundingBuilder = TimeZoneRounding.builder(TimeValue.parseTimeValue(interval, null)); tzRoundingBuilder = TimeZoneRounding.builder(TimeValue.parseTimeValue(interval, null, getClass().getSimpleName() + ".interval"));
} }
Rounding rounding = tzRoundingBuilder Rounding rounding = tzRoundingBuilder
@ -217,9 +217,9 @@ public class DateHistogramParser implements Aggregator.Parser {
private long parseOffset(String offset) throws IOException { private long parseOffset(String offset) throws IOException {
if (offset.charAt(0) == '-') { if (offset.charAt(0) == '-') {
return -TimeValue.parseTimeValue(offset.substring(1), null).millis(); return -TimeValue.parseTimeValue(offset.substring(1), null, getClass().getSimpleName() + ".parseOffset").millis();
} }
int beginIndex = offset.charAt(0) == '+' ? 1 : 0; int beginIndex = offset.charAt(0) == '+' ? 1 : 0;
return TimeValue.parseTimeValue(offset.substring(beginIndex), null).millis(); return TimeValue.parseTimeValue(offset.substring(beginIndex), null, getClass().getSimpleName() + ".parseOffset").millis();
} }
} }

View File

@ -106,7 +106,7 @@ public class DerivativeParser implements PipelineAggregator.Parser {
if (dateTimeUnit != null) { if (dateTimeUnit != null) {
xAxisUnits = dateTimeUnit.field().getDurationField().getUnitMillis(); xAxisUnits = dateTimeUnit.field().getDurationField().getUnitMillis();
} else { } else {
TimeValue timeValue = TimeValue.parseTimeValue(units, null); TimeValue timeValue = TimeValue.parseTimeValue(units, null, getClass().getSimpleName() + ".unit");
if (timeValue != null) { if (timeValue != null) {
xAxisUnits = timeValue.getMillis(); xAxisUnits = timeValue.getMillis();
} }

View File

@ -317,7 +317,7 @@ public class SearchSourceBuilder extends ToXContentToBytes {
* An optional timeout to control how long search is allowed to take. * An optional timeout to control how long search is allowed to take.
*/ */
public SearchSourceBuilder timeout(String timeout) { public SearchSourceBuilder timeout(String timeout) {
this.timeoutInMillis = TimeValue.parseTimeValue(timeout, null).millis(); this.timeoutInMillis = TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout").millis();
return this; return this;
} }

View File

@ -34,7 +34,7 @@ public class TimeoutParseElement implements SearchParseElement {
if (token == XContentParser.Token.VALUE_NUMBER) { if (token == XContentParser.Token.VALUE_NUMBER) {
context.timeoutInMillis(parser.longValue()); context.timeoutInMillis(parser.longValue());
} else { } else {
context.timeoutInMillis(TimeValue.parseTimeValue(parser.text(), null).millis()); context.timeoutInMillis(TimeValue.parseTimeValue(parser.text(), null, "timeout").millis());
} }
} }
} }

View File

@ -157,7 +157,16 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
final SnapshotId snapshotId = new SnapshotId(request.repository(), request.name()); final SnapshotId snapshotId = new SnapshotId(request.repository(), request.name());
final Snapshot snapshot = repository.readSnapshot(snapshotId); final Snapshot snapshot = repository.readSnapshot(snapshotId);
ImmutableList<String> filteredIndices = SnapshotUtils.filterIndices(snapshot.indices(), request.indices(), request.indicesOptions()); ImmutableList<String> filteredIndices = SnapshotUtils.filterIndices(snapshot.indices(), request.indices(), request.indicesOptions());
final MetaData metaData = repository.readSnapshotMetaData(snapshotId, filteredIndices); MetaData metaDataIn = repository.readSnapshotMetaData(snapshotId, filteredIndices);
final MetaData metaData;
if (snapshot.version().before(Version.V_2_0_0)) {
// ES 2.0 now requires units for all time and byte-sized settings, so we add the default unit if it's missing in this snapshot:
metaData = MetaData.addDefaultUnitsIfNeeded(logger, metaDataIn);
} else {
// Units are already enforced:
metaData = metaDataIn;
}
// Make sure that we can restore from this snapshot // Make sure that we can restore from this snapshot
validateSnapshotRestorable(snapshotId, snapshot); validateSnapshotRestorable(snapshotId, snapshot);

View File

@ -39,8 +39,8 @@ public class FsAppendBenchmark {
Path path = PathUtils.get("work/test.log"); Path path = PathUtils.get("work/test.log");
IOUtils.deleteFilesIgnoringExceptions(path); IOUtils.deleteFilesIgnoringExceptions(path);
int CHUNK = (int) ByteSizeValue.parseBytesSizeValue("1k").bytes(); int CHUNK = (int) ByteSizeValue.parseBytesSizeValue("1k", "CHUNK").bytes();
long DATA = ByteSizeValue.parseBytesSizeValue("10gb").bytes(); long DATA = ByteSizeValue.parseBytesSizeValue("10gb", "DATA").bytes();
byte[] data = new byte[CHUNK]; byte[] data = new byte[CHUNK];
new Random().nextBytes(data); new Random().nextBytes(data);

View File

@ -397,6 +397,6 @@ public class TermsAggregationSearchBenchmark {
totalQueryTime += searchResponse.getTookInMillis(); totalQueryTime += searchResponse.getTookInMillis();
} }
System.out.println("--> Terms stats agg (" + name + "): " + (totalQueryTime / QUERY_COUNT) + "ms"); System.out.println("--> Terms stats agg (" + name + "): " + (totalQueryTime / QUERY_COUNT) + "ms");
return new StatsResult(name, totalQueryTime, ByteSizeValue.parseBytesSizeValue("0b")); return new StatsResult(name, totalQueryTime, ByteSizeValue.parseBytesSizeValue("0b", "StatsResult"));
} }
} }

View File

@ -131,7 +131,8 @@ public class SimpleClusterStateTests extends ElasticsearchIntegrationTest {
@Test @Test
public void testLargeClusterStatePublishing() throws Exception { public void testLargeClusterStatePublishing() throws Exception {
int estimatedBytesSize = scaledRandomIntBetween(ByteSizeValue.parseBytesSizeValue("10k").bytesAsInt(), ByteSizeValue.parseBytesSizeValue("256k").bytesAsInt()); int estimatedBytesSize = scaledRandomIntBetween(ByteSizeValue.parseBytesSizeValue("10k", "estimatedBytesSize").bytesAsInt(),
ByteSizeValue.parseBytesSizeValue("256k", "estimatedBytesSize").bytesAsInt());
XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties"); XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties");
int counter = 0; int counter = 0;
int numberOfFields = 0; int numberOfFields = 0;

View File

@ -19,9 +19,6 @@
package org.elasticsearch.cluster.ack; package org.elasticsearch.cluster.ack;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse; import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesResponse; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesResponse;
@ -46,6 +43,11 @@ import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.warmer.IndexWarmersMetaData; import org.elasticsearch.search.warmer.IndexWarmersMetaData;
import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test; import org.junit.Test;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.cluster.metadata.IndexMetaData.*; import static org.elasticsearch.cluster.metadata.IndexMetaData.*;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -69,11 +71,11 @@ public class AckTests extends ElasticsearchIntegrationTest {
createIndex("test"); createIndex("test");
assertAcked(client().admin().indices().prepareUpdateSettings("test") assertAcked(client().admin().indices().prepareUpdateSettings("test")
.setSettings(Settings.builder().put("refresh_interval", 9999))); .setSettings(Settings.builder().put("refresh_interval", 9999, TimeUnit.MILLISECONDS)));
for (Client client : clients()) { for (Client client : clients()) {
String refreshInterval = getLocalClusterState(client).metaData().index("test").settings().get("index.refresh_interval"); String refreshInterval = getLocalClusterState(client).metaData().index("test").settings().get("index.refresh_interval");
assertThat(refreshInterval, equalTo("9999")); assertThat(refreshInterval, equalTo("9999ms"));
} }
} }
@ -81,7 +83,7 @@ public class AckTests extends ElasticsearchIntegrationTest {
public void testUpdateSettingsNoAcknowledgement() { public void testUpdateSettingsNoAcknowledgement() {
createIndex("test"); createIndex("test");
UpdateSettingsResponse updateSettingsResponse = client().admin().indices().prepareUpdateSettings("test").setTimeout("0s") UpdateSettingsResponse updateSettingsResponse = client().admin().indices().prepareUpdateSettings("test").setTimeout("0s")
.setSettings(Settings.builder().put("refresh_interval", 9999)).get(); .setSettings(Settings.builder().put("refresh_interval", 9999, TimeUnit.MILLISECONDS)).get();
assertThat(updateSettingsResponse.isAcknowledged(), equalTo(false)); assertThat(updateSettingsResponse.isAcknowledged(), equalTo(false));
} }

View File

@ -59,9 +59,9 @@ public class DiskThresholdDeciderUnitTests extends ElasticsearchTestCase {
}; };
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null); DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null);
assertThat(decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b"))); assertThat(decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test")));
assertThat(decider.getFreeDiskThresholdHigh(), equalTo(10.0d)); assertThat(decider.getFreeDiskThresholdHigh(), equalTo(10.0d));
assertThat(decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("0b"))); assertThat(decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test")));
assertThat(decider.getFreeDiskThresholdLow(), equalTo(15.0d)); assertThat(decider.getFreeDiskThresholdLow(), equalTo(15.0d));
assertThat(decider.getUsedDiskThresholdLow(), equalTo(85.0d)); assertThat(decider.getUsedDiskThresholdLow(), equalTo(85.0d));
assertThat(decider.getRerouteInterval().seconds(), equalTo(60L)); assertThat(decider.getRerouteInterval().seconds(), equalTo(60L));
@ -81,11 +81,11 @@ public class DiskThresholdDeciderUnitTests extends ElasticsearchTestCase {
applySettings.onRefreshSettings(newSettings); applySettings.onRefreshSettings(newSettings);
assertThat("high threshold bytes should be unset", assertThat("high threshold bytes should be unset",
decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b"))); decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test")));
assertThat("high threshold percentage should be changed", assertThat("high threshold percentage should be changed",
decider.getFreeDiskThresholdHigh(), equalTo(30.0d)); decider.getFreeDiskThresholdHigh(), equalTo(30.0d));
assertThat("low threshold bytes should be set to 500mb", assertThat("low threshold bytes should be set to 500mb",
decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("500mb"))); decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("500mb", "test")));
assertThat("low threshold bytes should be unset", assertThat("low threshold bytes should be unset",
decider.getFreeDiskThresholdLow(), equalTo(0.0d)); decider.getFreeDiskThresholdLow(), equalTo(0.0d));
assertThat("reroute interval should be changed to 30 seconds", assertThat("reroute interval should be changed to 30 seconds",

View File

@ -21,9 +21,13 @@ package org.elasticsearch.cluster.settings;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequestBuilder; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequestBuilder;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.indices.store.IndicesStore; import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest;
@ -62,7 +66,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
String key2 = DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION; String key2 = DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION;
boolean value2 = true; boolean value2 = true;
Settings transientSettings1 = Settings.builder().put(key1, value1).build(); Settings transientSettings1 = Settings.builder().put(key1, value1, ByteSizeUnit.BYTES).build();
Settings persistentSettings1 = Settings.builder().put(key2, value2).build(); Settings persistentSettings1 = Settings.builder().put(key2, value2).build();
ClusterUpdateSettingsResponse response1 = client().admin().cluster() ClusterUpdateSettingsResponse response1 = client().admin().cluster()
@ -78,7 +82,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
assertThat(response1.getPersistentSettings().get(key1), nullValue()); assertThat(response1.getPersistentSettings().get(key1), nullValue());
assertThat(response1.getPersistentSettings().get(key2), notNullValue()); assertThat(response1.getPersistentSettings().get(key2), notNullValue());
Settings transientSettings2 = Settings.builder().put(key1, value1).put(key2, value2).build(); Settings transientSettings2 = Settings.builder().put(key1, value1, ByteSizeUnit.BYTES).put(key2, value2).build();
Settings persistentSettings2 = Settings.EMPTY; Settings persistentSettings2 = Settings.EMPTY;
ClusterUpdateSettingsResponse response2 = client().admin().cluster() ClusterUpdateSettingsResponse response2 = client().admin().cluster()
@ -95,7 +99,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
assertThat(response2.getPersistentSettings().get(key2), nullValue()); assertThat(response2.getPersistentSettings().get(key2), nullValue());
Settings transientSettings3 = Settings.EMPTY; Settings transientSettings3 = Settings.EMPTY;
Settings persistentSettings3 = Settings.builder().put(key1, value1).put(key2, value2).build(); Settings persistentSettings3 = Settings.builder().put(key1, value1, ByteSizeUnit.BYTES).put(key2, value2).build();
ClusterUpdateSettingsResponse response3 = client().admin().cluster() ClusterUpdateSettingsResponse response3 = client().admin().cluster()
.prepareUpdateSettings() .prepareUpdateSettings()
@ -180,4 +184,39 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
assertThat(response.getPersistentSettings().get(key1), nullValue()); assertThat(response.getPersistentSettings().get(key1), nullValue());
assertThat(response.getPersistentSettings().get(key2), notNullValue()); assertThat(response.getPersistentSettings().get(key2), notNullValue());
} }
@Test(expected = IllegalArgumentException.class)
public void testMissingUnits() {
assertAcked(prepareCreate("test"));
// Should fail (missing units for refresh_interval):
client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder().put("index.refresh_interval", "10")).execute().actionGet();
}
@Test
public void testMissingUnitsLenient() {
try {
createNode(Settings.builder().put(Settings.SETTINGS_REQUIRE_UNITS, "false").build());
assertAcked(prepareCreate("test"));
ensureGreen();
client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder().put("index.refresh_interval", "10")).execute().actionGet();
} finally {
// Restore the default so subsequent tests require units:
assertFalse(Settings.getSettingsRequireUnits());
Settings.setSettingsRequireUnits(true);
}
}
private void createNode(Settings settings) {
internalCluster().startNode(Settings.builder()
.put(ClusterName.SETTING, "ClusterSettingsTests")
.put("node.name", "ClusterSettingsTests")
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
.put(EsExecutors.PROCESSORS, 1) // limit the number of threads created
.put("http.enabled", false)
.put("config.ignore_system_properties", true) // make sure we get what we set :)
.put(settings)
);
}
} }

View File

@ -24,9 +24,6 @@ import org.junit.Test;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.*;
/**
*
*/
public class SettingsValidatorTests extends ElasticsearchTestCase { public class SettingsValidatorTests extends ElasticsearchTestCase {
@Test @Test
@ -84,7 +81,7 @@ public class SettingsValidatorTests extends ElasticsearchTestCase {
assertThat(Validator.PERCENTAGE.validate("", "asdasd"), notNullValue()); assertThat(Validator.PERCENTAGE.validate("", "asdasd"), notNullValue());
assertThat(Validator.PERCENTAGE.validate("", "-1"), notNullValue()); assertThat(Validator.PERCENTAGE.validate("", "-1"), notNullValue());
assertThat(Validator.PERCENTAGE.validate("", "20"), notNullValue()); // we expect 20% assertThat(Validator.PERCENTAGE.validate("", "20"), notNullValue());
assertThat(Validator.PERCENTAGE.validate("", "-1%"), notNullValue()); assertThat(Validator.PERCENTAGE.validate("", "-1%"), notNullValue());
assertThat(Validator.PERCENTAGE.validate("", "101%"), notNullValue()); assertThat(Validator.PERCENTAGE.validate("", "101%"), notNullValue());
assertThat(Validator.PERCENTAGE.validate("", "100%"), nullValue()); assertThat(Validator.PERCENTAGE.validate("", "100%"), nullValue());
@ -92,7 +89,7 @@ public class SettingsValidatorTests extends ElasticsearchTestCase {
assertThat(Validator.PERCENTAGE.validate("", "0%"), nullValue()); assertThat(Validator.PERCENTAGE.validate("", "0%"), nullValue());
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "asdasd"), notNullValue()); assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "asdasd"), notNullValue());
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20"), nullValue()); assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20"), notNullValue());
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20mb"), nullValue()); assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20mb"), nullValue());
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "-1%"), notNullValue()); assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "-1%"), notNullValue());
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "101%"), notNullValue()); assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "101%"), notNullValue());

View File

@ -70,29 +70,79 @@ public class ByteSizeValueTests extends ElasticsearchTestCase {
@Test @Test
public void testParsing() { public void testParsing() {
assertThat(ByteSizeValue.parseBytesSizeValue("42pb").toString(), is("42pb")); assertThat(ByteSizeValue.parseBytesSizeValue("42PB", "testParsing").toString(), is("42pb"));
assertThat(ByteSizeValue.parseBytesSizeValue("42P").toString(), is("42pb")); assertThat(ByteSizeValue.parseBytesSizeValue("42 PB", "testParsing").toString(), is("42pb"));
assertThat(ByteSizeValue.parseBytesSizeValue("42PB").toString(), is("42pb")); assertThat(ByteSizeValue.parseBytesSizeValue("42pb", "testParsing").toString(), is("42pb"));
assertThat(ByteSizeValue.parseBytesSizeValue("54tb").toString(), is("54tb")); assertThat(ByteSizeValue.parseBytesSizeValue("42 pb", "testParsing").toString(), is("42pb"));
assertThat(ByteSizeValue.parseBytesSizeValue("54T").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("54TB").toString(), is("54tb")); assertThat(ByteSizeValue.parseBytesSizeValue("42P", "testParsing").toString(), is("42pb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12gb").toString(), is("12gb")); assertThat(ByteSizeValue.parseBytesSizeValue("42 P", "testParsing").toString(), is("42pb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12G").toString(), is("12gb")); assertThat(ByteSizeValue.parseBytesSizeValue("42p", "testParsing").toString(), is("42pb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12GB").toString(), is("12gb")); assertThat(ByteSizeValue.parseBytesSizeValue("42 p", "testParsing").toString(), is("42pb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12M").toString(), is("12mb"));
assertThat(ByteSizeValue.parseBytesSizeValue("1b").toString(), is("1b")); assertThat(ByteSizeValue.parseBytesSizeValue("54TB", "testParsing").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23kb").toString(), is("23kb")); assertThat(ByteSizeValue.parseBytesSizeValue("54 TB", "testParsing").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23k").toString(), is("23kb")); assertThat(ByteSizeValue.parseBytesSizeValue("54tb", "testParsing").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23").toString(), is("23b")); assertThat(ByteSizeValue.parseBytesSizeValue("54 tb", "testParsing").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("54T", "testParsing").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("54 T", "testParsing").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("54t", "testParsing").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("54 t", "testParsing").toString(), is("54tb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12GB", "testParsing").toString(), is("12gb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12 GB", "testParsing").toString(), is("12gb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12gb", "testParsing").toString(), is("12gb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12 gb", "testParsing").toString(), is("12gb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12G", "testParsing").toString(), is("12gb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12 G", "testParsing").toString(), is("12gb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12g", "testParsing").toString(), is("12gb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12 g", "testParsing").toString(), is("12gb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12M", "testParsing").toString(), is("12mb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12 M", "testParsing").toString(), is("12mb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12m", "testParsing").toString(), is("12mb"));
assertThat(ByteSizeValue.parseBytesSizeValue("12 m", "testParsing").toString(), is("12mb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23KB", "testParsing").toString(), is("23kb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23 KB", "testParsing").toString(), is("23kb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23kb", "testParsing").toString(), is("23kb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23 kb", "testParsing").toString(), is("23kb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23K", "testParsing").toString(), is("23kb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23 K", "testParsing").toString(), is("23kb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23k", "testParsing").toString(), is("23kb"));
assertThat(ByteSizeValue.parseBytesSizeValue("23 k", "testParsing").toString(), is("23kb"));
assertThat(ByteSizeValue.parseBytesSizeValue("1B", "testParsing").toString(), is("1b"));
assertThat(ByteSizeValue.parseBytesSizeValue("1 B", "testParsing").toString(), is("1b"));
assertThat(ByteSizeValue.parseBytesSizeValue("1b", "testParsing").toString(), is("1b"));
assertThat(ByteSizeValue.parseBytesSizeValue("1 b", "testParsing").toString(), is("1b"));
}
@Test(expected = ElasticsearchParseException.class)
public void testFailOnMissingUnits() {
ByteSizeValue.parseBytesSizeValue("23", "test");
}
@Test(expected = ElasticsearchParseException.class)
public void testFailOnUnknownUnits() {
ByteSizeValue.parseBytesSizeValue("23jw", "test");
} }
@Test(expected = ElasticsearchParseException.class) @Test(expected = ElasticsearchParseException.class)
public void testFailOnEmptyParsing() { public void testFailOnEmptyParsing() {
assertThat(ByteSizeValue.parseBytesSizeValue("").toString(), is("23kb")); assertThat(ByteSizeValue.parseBytesSizeValue("", "emptyParsing").toString(), is("23kb"));
} }
@Test(expected = ElasticsearchParseException.class) @Test(expected = ElasticsearchParseException.class)
public void testFailOnEmptyNumberParsing() { public void testFailOnEmptyNumberParsing() {
assertThat(ByteSizeValue.parseBytesSizeValue("g").toString(), is("23b")); assertThat(ByteSizeValue.parseBytesSizeValue("g", "emptyNumberParsing").toString(), is("23b"));
} }
}
@Test(expected = ElasticsearchParseException.class)
public void testNoDotsAllowed() {
ByteSizeValue.parseBytesSizeValue("42b.", null, "test");
}
}

View File

@ -132,7 +132,7 @@ public class FuzzinessTests extends ElasticsearchTestCase {
assertThat(parser.nextToken(), equalTo(XContentParser.Token.FIELD_NAME)); assertThat(parser.nextToken(), equalTo(XContentParser.Token.FIELD_NAME));
assertThat(parser.nextToken(), equalTo(XContentParser.Token.VALUE_STRING)); assertThat(parser.nextToken(), equalTo(XContentParser.Token.VALUE_STRING));
Fuzziness parse = Fuzziness.parse(parser); Fuzziness parse = Fuzziness.parse(parser);
assertThat(parse.asTimeValue(), equalTo(TimeValue.parseTimeValue(actual, null))); assertThat(parse.asTimeValue(), equalTo(TimeValue.parseTimeValue(actual, null, "fuzziness")));
assertThat(parser.nextToken(), equalTo(XContentParser.Token.END_OBJECT)); assertThat(parser.nextToken(), equalTo(XContentParser.Token.END_OBJECT));
} }
} }
@ -159,7 +159,7 @@ public class FuzzinessTests extends ElasticsearchTestCase {
assertThat(Fuzziness.AUTO.asDouble(), equalTo(1d)); assertThat(Fuzziness.AUTO.asDouble(), equalTo(1d));
assertThat(Fuzziness.AUTO.asLong(), equalTo(1l)); assertThat(Fuzziness.AUTO.asLong(), equalTo(1l));
assertThat(Fuzziness.AUTO.asShort(), equalTo((short) 1)); assertThat(Fuzziness.AUTO.asShort(), equalTo((short) 1));
assertThat(Fuzziness.AUTO.asTimeValue(), equalTo(TimeValue.parseTimeValue("1", TimeValue.timeValueMillis(1)))); assertThat(Fuzziness.AUTO.asTimeValue(), equalTo(TimeValue.parseTimeValue("1ms", TimeValue.timeValueMillis(1), "fuzziness")));
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.common.unit; package org.elasticsearch.common.unit;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.test.ElasticsearchTestCase;
@ -31,12 +32,8 @@ import java.util.concurrent.TimeUnit;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.lessThan;
/**
*
*/
public class TimeValueTests extends ElasticsearchTestCase { public class TimeValueTests extends ElasticsearchTestCase {
@Test
public void testSimple() { public void testSimple() {
assertThat(TimeUnit.MILLISECONDS.toMillis(10), equalTo(new TimeValue(10, TimeUnit.MILLISECONDS).millis())); assertThat(TimeUnit.MILLISECONDS.toMillis(10), equalTo(new TimeValue(10, TimeUnit.MILLISECONDS).millis()));
assertThat(TimeUnit.MICROSECONDS.toMicros(10), equalTo(new TimeValue(10, TimeUnit.MICROSECONDS).micros())); assertThat(TimeUnit.MICROSECONDS.toMicros(10), equalTo(new TimeValue(10, TimeUnit.MICROSECONDS).micros()));
@ -46,7 +43,6 @@ public class TimeValueTests extends ElasticsearchTestCase {
assertThat(TimeUnit.DAYS.toDays(10), equalTo(new TimeValue(10, TimeUnit.DAYS).days())); assertThat(TimeUnit.DAYS.toDays(10), equalTo(new TimeValue(10, TimeUnit.DAYS).days()));
} }
@Test
public void testToString() { public void testToString() {
assertThat("10ms", equalTo(new TimeValue(10, TimeUnit.MILLISECONDS).toString())); assertThat("10ms", equalTo(new TimeValue(10, TimeUnit.MILLISECONDS).toString()));
assertThat("1.5s", equalTo(new TimeValue(1533, TimeUnit.MILLISECONDS).toString())); assertThat("1.5s", equalTo(new TimeValue(1533, TimeUnit.MILLISECONDS).toString()));
@ -56,7 +52,6 @@ public class TimeValueTests extends ElasticsearchTestCase {
assertThat("1000d", equalTo(new TimeValue(1000, TimeUnit.DAYS).toString())); assertThat("1000d", equalTo(new TimeValue(1000, TimeUnit.DAYS).toString()));
} }
@Test
public void testFormat() { public void testFormat() {
assertThat(new TimeValue(1025, TimeUnit.MILLISECONDS).format(PeriodType.dayTime()), equalTo("1 second and 25 milliseconds")); assertThat(new TimeValue(1025, TimeUnit.MILLISECONDS).format(PeriodType.dayTime()), equalTo("1 second and 25 milliseconds"));
assertThat(new TimeValue(1, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("1 minute")); assertThat(new TimeValue(1, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("1 minute"));
@ -64,11 +59,67 @@ public class TimeValueTests extends ElasticsearchTestCase {
assertThat(new TimeValue(24 * 600 + 85, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("241 hours and 25 minutes")); assertThat(new TimeValue(24 * 600 + 85, TimeUnit.MINUTES).format(PeriodType.dayTime()), equalTo("241 hours and 25 minutes"));
} }
@Test
public void testMinusOne() { public void testMinusOne() {
assertThat(new TimeValue(-1).nanos(), lessThan(0l)); assertThat(new TimeValue(-1).nanos(), lessThan(0l));
} }
public void testParseTimeValue() {
// Space is allowed before unit:
assertEquals(new TimeValue(10, TimeUnit.MILLISECONDS),
TimeValue.parseTimeValue("10 ms", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.MILLISECONDS),
TimeValue.parseTimeValue("10ms", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.MILLISECONDS),
TimeValue.parseTimeValue("10 MS", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.MILLISECONDS),
TimeValue.parseTimeValue("10MS", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.SECONDS),
TimeValue.parseTimeValue("10 s", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.SECONDS),
TimeValue.parseTimeValue("10s", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.SECONDS),
TimeValue.parseTimeValue("10 S", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.SECONDS),
TimeValue.parseTimeValue("10S", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.MINUTES),
TimeValue.parseTimeValue("10 m", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.MINUTES),
TimeValue.parseTimeValue("10m", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.MINUTES),
TimeValue.parseTimeValue("10 M", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.MINUTES),
TimeValue.parseTimeValue("10M", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.HOURS),
TimeValue.parseTimeValue("10 h", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.HOURS),
TimeValue.parseTimeValue("10h", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.HOURS),
TimeValue.parseTimeValue("10 H", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.HOURS),
TimeValue.parseTimeValue("10H", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.DAYS),
TimeValue.parseTimeValue("10 d", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.DAYS),
TimeValue.parseTimeValue("10d", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.DAYS),
TimeValue.parseTimeValue("10 D", null, "test"));
assertEquals(new TimeValue(10, TimeUnit.DAYS),
TimeValue.parseTimeValue("10D", null, "test"));
assertEquals(new TimeValue(70, TimeUnit.DAYS),
TimeValue.parseTimeValue("10 w", null, "test"));
assertEquals(new TimeValue(70, TimeUnit.DAYS),
TimeValue.parseTimeValue("10w", null, "test"));
assertEquals(new TimeValue(70, TimeUnit.DAYS),
TimeValue.parseTimeValue("10 W", null, "test"));
assertEquals(new TimeValue(70, TimeUnit.DAYS),
TimeValue.parseTimeValue("10W", null, "test"));
}
private void assertEqualityAfterSerialize(TimeValue value) throws IOException { private void assertEqualityAfterSerialize(TimeValue value) throws IOException {
BytesStreamOutput out = new BytesStreamOutput(); BytesStreamOutput out = new BytesStreamOutput();
value.writeTo(out); value.writeTo(out);
@ -79,11 +130,24 @@ public class TimeValueTests extends ElasticsearchTestCase {
assertThat(inValue, equalTo(value)); assertThat(inValue, equalTo(value));
} }
@Test
public void testSerialize() throws Exception { public void testSerialize() throws Exception {
assertEqualityAfterSerialize(new TimeValue(100, TimeUnit.DAYS)); assertEqualityAfterSerialize(new TimeValue(100, TimeUnit.DAYS));
assertEqualityAfterSerialize(new TimeValue(-1)); assertEqualityAfterSerialize(new TimeValue(-1));
assertEqualityAfterSerialize(new TimeValue(1, TimeUnit.NANOSECONDS)); assertEqualityAfterSerialize(new TimeValue(1, TimeUnit.NANOSECONDS));
}
@Test(expected = ElasticsearchParseException.class)
public void testFailOnUnknownUnits() {
TimeValue.parseTimeValue("23tw", null, "test");
}
@Test(expected = ElasticsearchParseException.class)
public void testFailOnMissingUnits() {
TimeValue.parseTimeValue("42", null, "test");
}
@Test(expected = ElasticsearchParseException.class)
public void testNoDotsAllowed() {
TimeValue.parseTimeValue("42ms.", null, "test");
} }
} }

View File

@ -24,6 +24,7 @@ import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
@ -336,7 +337,7 @@ public class BigArraysTests extends ElasticsearchSingleNodeTest {
for (String type : Arrays.asList("Byte", "Int", "Long", "Float", "Double", "Object")) { for (String type : Arrays.asList("Byte", "Int", "Long", "Float", "Double", "Object")) {
HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService( HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService(
Settings.builder() Settings.builder()
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, size - 1) .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, size - 1, ByteSizeUnit.BYTES)
.build(), .build(),
new NodeSettingsService(Settings.EMPTY)); new NodeSettingsService(Settings.EMPTY));
BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking(); BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();
@ -356,7 +357,7 @@ public class BigArraysTests extends ElasticsearchSingleNodeTest {
final long maxSize = randomIntBetween(1 << 10, 1 << 22); final long maxSize = randomIntBetween(1 << 10, 1 << 22);
HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService( HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService(
Settings.builder() Settings.builder()
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, maxSize) .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, maxSize, ByteSizeUnit.BYTES)
.build(), .build(),
new NodeSettingsService(Settings.EMPTY)); new NodeSettingsService(Settings.EMPTY));
BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking(); BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();

View File

@ -23,6 +23,8 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.test.ElasticsearchSingleNodeTest; import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import java.util.concurrent.TimeUnit;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
@ -47,7 +49,11 @@ public class InternalEngineSettingsTest extends ElasticsearchSingleNodeTest {
final int iters = between(1, 20); final int iters = between(1, 20);
for (int i = 0; i < iters; i++) { for (int i = 0; i < iters; i++) {
boolean compoundOnFlush = randomBoolean(); boolean compoundOnFlush = randomBoolean();
long gcDeletes = Math.max(0, randomLong());
// Tricky: TimeValue.parseTimeValue casts this long to a double, which steals 11 of the 64 bits for exponent, so we can't use
// the full long range here else the assert below fails:
long gcDeletes = random().nextLong() & (Long.MAX_VALUE >> 11);
boolean versionMapAsPercent = randomBoolean(); boolean versionMapAsPercent = randomBoolean();
double versionMapPercent = randomIntBetween(0, 100); double versionMapPercent = randomIntBetween(0, 100);
long versionMapSizeInMB = randomIntBetween(10, 20); long versionMapSizeInMB = randomIntBetween(10, 20);
@ -55,9 +61,10 @@ public class InternalEngineSettingsTest extends ElasticsearchSingleNodeTest {
Settings build = Settings.builder() Settings build = Settings.builder()
.put(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush) .put(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush)
.put(EngineConfig.INDEX_GC_DELETES_SETTING, gcDeletes) .put(EngineConfig.INDEX_GC_DELETES_SETTING, gcDeletes, TimeUnit.MILLISECONDS)
.put(EngineConfig.INDEX_VERSION_MAP_SIZE, versionMapString) .put(EngineConfig.INDEX_VERSION_MAP_SIZE, versionMapString)
.build(); .build();
assertEquals(gcDeletes, build.getAsTime(EngineConfig.INDEX_GC_DELETES_SETTING, null).millis());
client().admin().indices().prepareUpdateSettings("foo").setSettings(build).get(); client().admin().indices().prepareUpdateSettings("foo").setSettings(build).get();
LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig(); LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig();
@ -78,7 +85,7 @@ public class InternalEngineSettingsTest extends ElasticsearchSingleNodeTest {
} }
Settings settings = Settings.builder() Settings settings = Settings.builder()
.put(EngineConfig.INDEX_GC_DELETES_SETTING, 1000) .put(EngineConfig.INDEX_GC_DELETES_SETTING, 1000, TimeUnit.MILLISECONDS)
.build(); .build();
client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get(); client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get();
assertEquals(engine.getGcDeletesInMillis(), 1000); assertEquals(engine.getGcDeletesInMillis(), 1000);
@ -94,7 +101,7 @@ public class InternalEngineSettingsTest extends ElasticsearchSingleNodeTest {
assertTrue(engine.config().isEnableGcDeletes()); assertTrue(engine.config().isEnableGcDeletes());
settings = Settings.builder() settings = Settings.builder()
.put(EngineConfig.INDEX_GC_DELETES_SETTING, 1000) .put(EngineConfig.INDEX_GC_DELETES_SETTING, 1000, TimeUnit.MILLISECONDS)
.build(); .build();
client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get(); client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get();
assertEquals(engine.getGcDeletesInMillis(), 1000); assertEquals(engine.getGcDeletesInMillis(), 1000);

View File

@ -48,6 +48,7 @@ import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.gateway.GatewayAllocator;
import org.elasticsearch.index.merge.policy.MergePolicyModule; import org.elasticsearch.index.merge.policy.MergePolicyModule;
@ -489,7 +490,7 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
.setType("fs").setSettings(settingsBuilder() .setType("fs").setSettings(settingsBuilder()
.put("location", randomRepoPath().toAbsolutePath()) .put("location", randomRepoPath().toAbsolutePath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
logger.info("--> snapshot"); logger.info("--> snapshot");
CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test").get(); CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test").get();
assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.PARTIAL)); assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.PARTIAL));

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.translog; package org.elasticsearch.index.translog;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
import java.io.IOException; import java.io.IOException;
@ -34,7 +35,7 @@ public class BufferedTranslogTests extends TranslogTests {
protected Translog create(Path path) throws IOException { protected Translog create(Path path) throws IOException {
Settings build = Settings.settingsBuilder() Settings build = Settings.settingsBuilder()
.put("index.translog.fs.type", TranslogWriter.Type.BUFFERED.name()) .put("index.translog.fs.type", TranslogWriter.Type.BUFFERED.name())
.put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024)) .put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024), ByteSizeUnit.BYTES)
.build(); .build();
TranslogConfig translogConfig = new TranslogConfig(shardId, path, build, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null); TranslogConfig translogConfig = new TranslogConfig(shardId, path, build, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
return new Translog(translogConfig); return new Translog(translogConfig);

View File

@ -38,6 +38,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.discovery.DiscoveryService; import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.index.recovery.RecoveryStats; import org.elasticsearch.index.recovery.RecoveryStats;
@ -47,8 +48,8 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.RecoveryState.Stage; import org.elasticsearch.indices.recovery.RecoveryState.Stage;
import org.elasticsearch.indices.recovery.RecoveryState.Type; import org.elasticsearch.indices.recovery.RecoveryState.Type;
import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.snapshots.SnapshotState;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.test.store.MockFSDirectoryService;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
@ -126,8 +127,8 @@ public class IndexRecoveryTests extends ElasticsearchIntegrationTest {
assertTrue(client().admin().cluster().prepareUpdateSettings() assertTrue(client().admin().cluster().prepareUpdateSettings()
.setTransientSettings(Settings.builder() .setTransientSettings(Settings.builder()
// one chunk per sec.. // one chunk per sec..
.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize) .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize, ByteSizeUnit.BYTES)
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize) .put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize, ByteSizeUnit.BYTES)
) )
.get().isAcknowledged()); .get().isAcknowledged());
} }

View File

@ -56,7 +56,8 @@ public class DirectBufferNetworkTests extends ElasticsearchIntegrationTest {
public void verifySaneDirectBufferAllocations() throws Exception { public void verifySaneDirectBufferAllocations() throws Exception {
createIndex("test"); createIndex("test");
int estimatedBytesSize = scaledRandomIntBetween(ByteSizeValue.parseBytesSizeValue("1.1mb").bytesAsInt(), ByteSizeValue.parseBytesSizeValue("1.5mb").bytesAsInt()); int estimatedBytesSize = scaledRandomIntBetween(ByteSizeValue.parseBytesSizeValue("1.1mb", "estimatedBytesSize").bytesAsInt(),
ByteSizeValue.parseBytesSizeValue("1.5mb", "estimatedBytesSize").bytesAsInt());
byte[] data = new byte[estimatedBytesSize]; byte[] data = new byte[estimatedBytesSize];
getRandom().nextBytes(data); getRandom().nextBytes(data);

View File

@ -59,7 +59,7 @@ public class TTLPercolatorTests extends ElasticsearchIntegrationTest {
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return settingsBuilder() return settingsBuilder()
.put(super.nodeSettings(nodeOrdinal)) .put(super.nodeSettings(nodeOrdinal))
.put("indices.ttl.interval", PURGE_INTERVAL) .put("indices.ttl.interval", PURGE_INTERVAL, TimeUnit.MILLISECONDS)
.build(); .build();
} }
@ -163,7 +163,7 @@ public class TTLPercolatorTests extends ElasticsearchIntegrationTest {
public void testEnsureTTLDoesNotCreateIndex() throws IOException, InterruptedException { public void testEnsureTTLDoesNotCreateIndex() throws IOException, InterruptedException {
ensureGreen(); ensureGreen();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder() client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder()
.put("indices.ttl.interval", 60) // 60 sec .put("indices.ttl.interval", 60, TimeUnit.SECONDS) // 60 sec
.build()).get(); .build()).get();
String typeMapping = XContentFactory.jsonBuilder().startObject().startObject("type1") String typeMapping = XContentFactory.jsonBuilder().startObject().startObject("type1")
@ -176,7 +176,7 @@ public class TTLPercolatorTests extends ElasticsearchIntegrationTest {
.execute().actionGet(); .execute().actionGet();
ensureGreen(); ensureGreen();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder() client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder()
.put("indices.ttl.interval", 1) // 60 sec .put("indices.ttl.interval", 1, TimeUnit.SECONDS)
.build()).get(); .build()).get();
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {

View File

@ -19,10 +19,13 @@
package org.elasticsearch.recovery; package org.elasticsearch.recovery;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.test.ElasticsearchSingleNodeTest; import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import org.junit.Test; import org.junit.Test;
import java.util.concurrent.TimeUnit;
public class RecoverySettingsTest extends ElasticsearchSingleNodeTest { public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
@Override @Override
@ -32,7 +35,7 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
@Test @Test
public void testAllSettingsAreDynamicallyUpdatable() { public void testAllSettingsAreDynamicallyUpdatable() {
innerTestSettings(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, randomIntBetween(1, 200), new Validator() { innerTestSettings(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, randomIntBetween(1, 200), ByteSizeUnit.BYTES, new Validator() {
@Override @Override
public void validate(RecoverySettings recoverySettings, int expectedValue) { public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.fileChunkSize().bytesAsInt()); assertEquals(expectedValue, recoverySettings.fileChunkSize().bytesAsInt());
@ -44,7 +47,7 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
assertEquals(expectedValue, recoverySettings.translogOps()); assertEquals(expectedValue, recoverySettings.translogOps());
} }
}); });
innerTestSettings(RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE, randomIntBetween(1, 200), new Validator() { innerTestSettings(RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE, randomIntBetween(1, 200), ByteSizeUnit.BYTES, new Validator() {
@Override @Override
public void validate(RecoverySettings recoverySettings, int expectedValue) { public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.translogSize().bytesAsInt()); assertEquals(expectedValue, recoverySettings.translogSize().bytesAsInt());
@ -68,31 +71,31 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
assertEquals(null, recoverySettings.rateLimiter()); assertEquals(null, recoverySettings.rateLimiter());
} }
}); });
innerTestSettings(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC, randomIntBetween(1, 200), new Validator() { innerTestSettings(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC, randomIntBetween(1, 200), TimeUnit.MILLISECONDS, new Validator() {
@Override @Override
public void validate(RecoverySettings recoverySettings, int expectedValue) { public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.retryDelayStateSync().millis()); assertEquals(expectedValue, recoverySettings.retryDelayStateSync().millis());
} }
}); });
innerTestSettings(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK, randomIntBetween(1, 200), new Validator() { innerTestSettings(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK, randomIntBetween(1, 200), TimeUnit.MILLISECONDS, new Validator() {
@Override @Override
public void validate(RecoverySettings recoverySettings, int expectedValue) { public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.retryDelayNetwork().millis()); assertEquals(expectedValue, recoverySettings.retryDelayNetwork().millis());
} }
}); });
innerTestSettings(RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT, randomIntBetween(1, 200), new Validator() { innerTestSettings(RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT, randomIntBetween(1, 200), TimeUnit.MILLISECONDS, new Validator() {
@Override @Override
public void validate(RecoverySettings recoverySettings, int expectedValue) { public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.activityTimeout().millis()); assertEquals(expectedValue, recoverySettings.activityTimeout().millis());
} }
}); });
innerTestSettings(RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT, randomIntBetween(1, 200), new Validator() { innerTestSettings(RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT, randomIntBetween(1, 200), TimeUnit.MILLISECONDS, new Validator() {
@Override @Override
public void validate(RecoverySettings recoverySettings, int expectedValue) { public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.internalActionTimeout().millis()); assertEquals(expectedValue, recoverySettings.internalActionTimeout().millis());
} }
}); });
innerTestSettings(RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT, randomIntBetween(1, 200), new Validator() { innerTestSettings(RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT, randomIntBetween(1, 200), TimeUnit.MILLISECONDS, new Validator() {
@Override @Override
public void validate(RecoverySettings recoverySettings, int expectedValue) { public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.internalActionLongTimeout().millis()); assertEquals(expectedValue, recoverySettings.internalActionLongTimeout().millis());
@ -120,6 +123,16 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
validator.validate(getInstanceFromNode(RecoverySettings.class), newValue); validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);
} }
private void innerTestSettings(String key, int newValue, TimeUnit timeUnit, Validator validator) {
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(key, newValue, timeUnit)).get();
validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);
}
private void innerTestSettings(String key, int newValue, ByteSizeUnit byteSizeUnit, Validator validator) {
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(key, newValue, byteSizeUnit)).get();
validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);
}
private void innerTestSettings(String key, boolean newValue, Validator validator) { private void innerTestSettings(String key, boolean newValue, Validator validator) {
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(key, newValue)).get(); client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(key, newValue)).get();
validator.validate(getInstanceFromNode(RecoverySettings.class), newValue); validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);

View File

@ -43,6 +43,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDeci
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.discovery.DiscoveryService; import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeEnvironment;
@ -56,8 +57,8 @@ import org.elasticsearch.indices.recovery.RecoveryTarget;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.test.BackgroundIndexer;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.*; import org.elasticsearch.transport.*;
@ -397,8 +398,8 @@ public class RelocationTests extends ElasticsearchIntegrationTest {
assertTrue(client().admin().cluster().prepareUpdateSettings() assertTrue(client().admin().cluster().prepareUpdateSettings()
.setTransientSettings(Settings.builder() .setTransientSettings(Settings.builder()
// one chunk per sec.. // one chunk per sec..
.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize) .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize, ByteSizeUnit.BYTES)
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize) .put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize, ByteSizeUnit.BYTES)
) )
.get().isAcknowledged()); .get().isAcknowledged());

View File

@ -1304,14 +1304,14 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
/** /**
* see issue #9634, negative interval in date_histogram should raise exception * see issue #9634, negative interval in date_histogram should raise exception
*/ */
public void testExeptionOnNegativerInterval() { public void testExceptionOnNegativeInterval() {
try { try {
client().prepareSearch("idx") client().prepareSearch("idx")
.addAggregation(dateHistogram("histo").field("date").interval(-TimeUnit.DAYS.toMillis(1)).minDocCount(0)).execute() .addAggregation(dateHistogram("histo").field("date").interval(-TimeUnit.DAYS.toMillis(1)).minDocCount(0)).execute()
.actionGet(); .actionGet();
fail(); fail();
} catch (SearchPhaseExecutionException e) { } catch (SearchPhaseExecutionException e) {
assertThat(e.toString(), containsString("IllegalArgumentException")); assertThat(e.toString(), containsString("ElasticsearchParseException"));
} }
} }

View File

@ -47,7 +47,7 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.core.IsNull.notNullValue; import static org.hamcrest.core.IsNull.notNullValue;
@ElasticsearchIntegrationTest.SuiteScopeTest @ElasticsearchIntegrationTest.SuiteScopeTest
public class TermsDocCountErrorTests extends ElasticsearchIntegrationTest{ public class TermsDocCountErrorTests extends ElasticsearchIntegrationTest {
private static final String STRING_FIELD_NAME = "s_value"; private static final String STRING_FIELD_NAME = "s_value";
private static final String LONG_FIELD_NAME = "l_value"; private static final String LONG_FIELD_NAME = "l_value";

View File

@ -211,8 +211,8 @@ public class DerivativeTests extends ElasticsearchIntegrationTest {
.prepareSearch("idx") .prepareSearch("idx")
.addAggregation( .addAggregation(
histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0)
.subAggregation(derivative("deriv").setBucketsPaths("_count").unit("1")) .subAggregation(derivative("deriv").setBucketsPaths("_count").unit("1ms"))
.subAggregation(derivative("2nd_deriv").setBucketsPaths("deriv").unit("10"))).execute().actionGet(); .subAggregation(derivative("2nd_deriv").setBucketsPaths("deriv").unit("10ms"))).execute().actionGet();
assertSearchResponse(response); assertSearchResponse(response);

View File

@ -499,7 +499,7 @@ public class SearchScrollTests extends ElasticsearchIntegrationTest {
RestSearchScrollAction.buildFromContent(content, searchScrollRequest); RestSearchScrollAction.buildFromContent(content, searchScrollRequest);
assertThat(searchScrollRequest.scrollId(), equalTo("SCROLL_ID")); assertThat(searchScrollRequest.scrollId(), equalTo("SCROLL_ID"));
assertThat(searchScrollRequest.scroll().keepAlive(), equalTo(TimeValue.parseTimeValue("1m", null))); assertThat(searchScrollRequest.scroll().keepAlive(), equalTo(TimeValue.parseTimeValue("1m", null, "scroll")));
} }
@Test @Test

View File

@ -24,6 +24,7 @@ import com.carrotsearch.hppc.IntSet;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFuture;
import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.ListenableActionFuture; import org.elasticsearch.action.ListenableActionFuture;
import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryResponse; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryResponse;
@ -36,20 +37,21 @@ import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusRe
import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask; import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.MetaData.Custom; import org.elasticsearch.cluster.metadata.MetaData.Custom;
import org.elasticsearch.cluster.metadata.SnapshotMetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
@ -706,7 +708,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
int initialIndices = between(1, 3); int initialIndices = between(1, 3);
logger.info("--> create {} indices", initialIndices); logger.info("--> create {} indices", initialIndices);
@ -813,7 +815,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
assertAcked(prepareCreate("test-idx", 0, settingsBuilder().put("number_of_shards", between(1, 20)) assertAcked(prepareCreate("test-idx", 0, settingsBuilder().put("number_of_shards", between(1, 20))
.put("number_of_replicas", 0))); .put("number_of_replicas", 0)));

View File

@ -31,6 +31,7 @@ import org.elasticsearch.cluster.metadata.RepositoriesMetaData;
import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData;
import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.repositories.RepositoryException; import org.elasticsearch.repositories.RepositoryException;
import org.elasticsearch.repositories.RepositoryVerificationException; import org.elasticsearch.repositories.RepositoryVerificationException;
import org.elasticsearch.snapshots.mockstore.MockRepositoryModule; import org.elasticsearch.snapshots.mockstore.MockRepositoryModule;
@ -155,7 +156,7 @@ public class RepositoriesTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(5, 100)) .put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES)
) )
.setTimeout("0s").get(); .setTimeout("0s").get();
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(false)); assertThat(putRepositoryResponse.isAcknowledged(), equalTo(false));
@ -165,7 +166,7 @@ public class RepositoriesTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(5, 100)) .put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES)
).get(); ).get();
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true)); assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));

View File

@ -50,6 +50,7 @@ import org.elasticsearch.cluster.service.PendingClusterTask;
import org.elasticsearch.common.Priority; import org.elasticsearch.common.Priority;
import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.IndexStore;
@ -91,7 +92,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
createIndex("test-idx-1", "test-idx-2", "test-idx-3"); createIndex("test-idx-1", "test-idx-2", "test-idx-3");
ensureGreen(); ensureGreen();
@ -232,7 +233,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
createIndex("test"); createIndex("test");
String originalIndexUUID = client().admin().indices().prepareGetSettings("test").get().getSetting("test", IndexMetaData.SETTING_UUID); String originalIndexUUID = client().admin().indices().prepareGetSettings("test").get().getSetting("test", IndexMetaData.SETTING_UUID);
@ -281,11 +282,11 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
logger.info("--> create index with foo type"); logger.info("--> create index with foo type");
assertAcked(prepareCreate("test-idx", 2, Settings.builder() assertAcked(prepareCreate("test-idx", 2, Settings.builder()
.put(indexSettings()).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)).put("refresh_interval", 10))); .put(indexSettings()).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)).put("refresh_interval", 10, TimeUnit.SECONDS)));
NumShards numShards = getNumShards("test-idx"); NumShards numShards = getNumShards("test-idx");
@ -300,7 +301,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
logger.info("--> delete the index and recreate it with bar type"); logger.info("--> delete the index and recreate it with bar type");
cluster().wipeIndices("test-idx"); cluster().wipeIndices("test-idx");
assertAcked(prepareCreate("test-idx", 2, Settings.builder() assertAcked(prepareCreate("test-idx", 2, Settings.builder()
.put(SETTING_NUMBER_OF_SHARDS, numShards.numPrimaries).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)).put("refresh_interval", 5))); .put(SETTING_NUMBER_OF_SHARDS, numShards.numPrimaries).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)).put("refresh_interval", 5, TimeUnit.SECONDS)));
assertAcked(client().admin().indices().preparePutMapping("test-idx").setType("bar").setSource("baz", "type=string")); assertAcked(client().admin().indices().preparePutMapping("test-idx").setType("bar").setSource("baz", "type=string"));
ensureGreen(); ensureGreen();
@ -318,7 +319,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
logger.info("--> assert that old settings are restored"); logger.info("--> assert that old settings are restored");
GetSettingsResponse getSettingsResponse = client.admin().indices().prepareGetSettings("test-idx").execute().actionGet(); GetSettingsResponse getSettingsResponse = client.admin().indices().prepareGetSettings("test-idx").execute().actionGet();
assertThat(getSettingsResponse.getSetting("test-idx", "index.refresh_interval"), equalTo("10")); assertThat(getSettingsResponse.getSetting("test-idx", "index.refresh_interval"), equalTo("10000ms"));
} }
@Test @Test
@ -761,7 +762,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", repo) .put("location", repo)
.put("compress", false) .put("compress", false)
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
createIndex("test-idx"); createIndex("test-idx");
ensureGreen(); ensureGreen();
@ -818,7 +819,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", repo) .put("location", repo)
.put("compress", false) .put("compress", false)
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
createIndex("test-idx-1", "test-idx-2"); createIndex("test-idx-1", "test-idx-2");
ensureYellow(); ensureYellow();
@ -857,7 +858,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", repo) .put("location", repo)
.put("compress", false) .put("compress", false)
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
createIndex("test-idx-1", "test-idx-2"); createIndex("test-idx-1", "test-idx-2");
ensureYellow(); ensureYellow();
@ -892,7 +893,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", repo) .put("location", repo)
.put("compress", false) .put("compress", false)
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
createIndex("test-idx-1", "test-idx-2"); createIndex("test-idx-1", "test-idx-2");
ensureYellow(); ensureYellow();
@ -1247,7 +1248,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", repositoryLocation) .put("location", repositoryLocation)
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
createIndex("test-idx"); createIndex("test-idx");
ensureGreen(); ensureGreen();
@ -1307,7 +1308,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", repositoryLocation) .put("location", repositoryLocation)
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(1000, 10000)) .put("chunk_size", randomIntBetween(1000, 10000), ByteSizeUnit.BYTES)
.put("max_restore_bytes_per_sec", throttleRestore ? "0.5k" : "0") .put("max_restore_bytes_per_sec", throttleRestore ? "0.5k" : "0")
.put("max_snapshot_bytes_per_sec", throttleSnapshot ? "0.5k" : "0"))); .put("max_snapshot_bytes_per_sec", throttleSnapshot ? "0.5k" : "0")));
@ -1463,7 +1464,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
// Create index on 1 nodes and make sure each node has a primary by setting no replicas // Create index on 1 nodes and make sure each node has a primary by setting no replicas
assertAcked(prepareCreate("test-idx", 1, Settings.builder().put("number_of_replicas", 0))); assertAcked(prepareCreate("test-idx", 1, Settings.builder().put("number_of_replicas", 0)));
@ -1478,7 +1479,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
// Update settings to make sure that relocation is slow so we can start snapshot before relocation is finished // Update settings to make sure that relocation is slow so we can start snapshot before relocation is finished
assertAcked(client.admin().indices().prepareUpdateSettings("test-idx").setSettings(Settings.builder() assertAcked(client.admin().indices().prepareUpdateSettings("test-idx").setSettings(Settings.builder()
.put(IndexStore.INDEX_STORE_THROTTLE_TYPE, "all") .put(IndexStore.INDEX_STORE_THROTTLE_TYPE, "all")
.put(IndexStore.INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC, 100) .put(IndexStore.INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC, 100, ByteSizeUnit.BYTES)
)); ));
logger.info("--> start relocations"); logger.info("--> start relocations");
@ -1511,7 +1512,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
// only one shard // only one shard
assertAcked(prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1))); assertAcked(prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)));
@ -1574,7 +1575,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
logger.info("--> create test index with synonyms search analyzer"); logger.info("--> create test index with synonyms search analyzer");
@ -1691,7 +1692,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType(MockRepositoryModule.class.getCanonicalName()).setSettings(Settings.settingsBuilder() .setType(MockRepositoryModule.class.getCanonicalName()).setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)) .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
.put("block_on_init", true) .put("block_on_init", true)
)); ));
@ -1743,7 +1744,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType(MockRepositoryModule.class.getCanonicalName()).setSettings(Settings.settingsBuilder() .setType(MockRepositoryModule.class.getCanonicalName()).setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)) .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
)); ));
createIndex("test-idx"); createIndex("test-idx");
@ -1823,7 +1824,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath()) .put("location", randomRepoPath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
assertAcked(prepareCreate("test-idx", 0, settingsBuilder().put("number_of_shards", between(1, 20)) assertAcked(prepareCreate("test-idx", 0, settingsBuilder().put("number_of_shards", between(1, 20))
.put("number_of_replicas", 0))); .put("number_of_replicas", 0)));

View File

@ -32,6 +32,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.test.ElasticsearchBackwardsCompatIntegrationTest; import org.elasticsearch.test.ElasticsearchBackwardsCompatIntegrationTest;
import org.junit.Test; import org.junit.Test;
@ -58,7 +59,7 @@ public class SnapshotBackwardsCompatibilityTest extends ElasticsearchBackwardsCo
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", randomRepoPath().toAbsolutePath()) .put("location", randomRepoPath().toAbsolutePath())
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
String[] indicesBefore = new String[randomIntBetween(2,5)]; String[] indicesBefore = new String[randomIntBetween(2,5)];
String[] indicesAfter = new String[randomIntBetween(2,5)]; String[] indicesAfter = new String[randomIntBetween(2,5)];
for (int i = 0; i < indicesBefore.length; i++) { for (int i = 0; i < indicesBefore.length; i++) {
@ -168,7 +169,7 @@ public class SnapshotBackwardsCompatibilityTest extends ElasticsearchBackwardsCo
.setType("fs").setSettings(Settings.settingsBuilder() .setType("fs").setSettings(Settings.settingsBuilder()
.put("location", tempDir) .put("location", tempDir)
.put("compress", randomBoolean()) .put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000)))); .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
// only one shard // only one shard
assertAcked(prepareCreate("test").setSettings(Settings.builder() assertAcked(prepareCreate("test").setSettings(Settings.builder()

View File

@ -39,7 +39,7 @@ public class GetStressTest {
final int NUMBER_OF_NODES = 2; final int NUMBER_OF_NODES = 2;
final int NUMBER_OF_THREADS = 50; final int NUMBER_OF_THREADS = 50;
final TimeValue TEST_TIME = TimeValue.parseTimeValue("10m", null); final TimeValue TEST_TIME = TimeValue.parseTimeValue("10m", null, "TEST_TIME");
Node[] nodes = new Node[NUMBER_OF_NODES]; Node[] nodes = new Node[NUMBER_OF_NODES];
for (int i = 0; i < nodes.length; i++) { for (int i = 0; i < nodes.length; i++) {
@ -93,4 +93,4 @@ public class GetStressTest {
System.out.println("test done."); System.out.println("test done.");
done.set(true); done.set(true);
} }
} }

View File

@ -448,7 +448,7 @@ public final class InternalTestCluster extends TestCluster {
if (rarely(random)) { if (rarely(random)) {
builder.put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, 0); // 0 has special meaning to sync each op builder.put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, 0); // 0 has special meaning to sync each op
} else { } else {
builder.put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, RandomInts.randomIntBetween(random, 100, 5000)); builder.put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, RandomInts.randomIntBetween(random, 100, 5000), TimeUnit.MILLISECONDS);
} }
} }

View File

@ -490,7 +490,7 @@ public abstract class AbstractSimpleTransportTests extends ElasticsearchTestCase
serviceA.registerRequestHandler("sayHelloTimeoutDelayedResponse", StringMessageRequest.class, ThreadPool.Names.GENERIC, new TransportRequestHandler<StringMessageRequest>() { serviceA.registerRequestHandler("sayHelloTimeoutDelayedResponse", StringMessageRequest.class, ThreadPool.Names.GENERIC, new TransportRequestHandler<StringMessageRequest>() {
@Override @Override
public void messageReceived(StringMessageRequest request, TransportChannel channel) { public void messageReceived(StringMessageRequest request, TransportChannel channel) {
TimeValue sleep = TimeValue.parseTimeValue(request.message, null); TimeValue sleep = TimeValue.parseTimeValue(request.message, null, "sleep");
try { try {
Thread.sleep(sleep.millis()); Thread.sleep(sleep.millis());
} catch (InterruptedException e) { } catch (InterruptedException e) {

View File

@ -57,7 +57,7 @@ public class SimpleTTLTests extends ElasticsearchIntegrationTest {
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return settingsBuilder() return settingsBuilder()
.put(super.nodeSettings(nodeOrdinal)) .put(super.nodeSettings(nodeOrdinal))
.put("indices.ttl.interval", PURGE_INTERVAL) .put("indices.ttl.interval", PURGE_INTERVAL, TimeUnit.MILLISECONDS)
.put("cluster.routing.operation.use_type", false) // make sure we control the shard computation .put("cluster.routing.operation.use_type", false) // make sure we control the shard computation
.put("cluster.routing.operation.hash.type", "djb") .put("cluster.routing.operation.hash.type", "djb")
.build(); .build();

View File

@ -217,6 +217,21 @@ public class SimpleVersioningTests extends ElasticsearchIntegrationTest {
assertThat(indexResponse.getVersion(), equalTo(20l)); assertThat(indexResponse.getVersion(), equalTo(20l));
} }
@Test
public void testRequireUnitsOnUpdateSettings() throws Exception {
createIndex("test");
ensureGreen();
HashMap<String,Object> newSettings = new HashMap<>();
newSettings.put("index.gc_deletes", "42");
try {
client().admin().indices().prepareUpdateSettings("test").setSettings(newSettings).execute().actionGet();
fail("did not hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
assertTrue(iae.getMessage().contains("Failed to parse setting [index.gc_deletes] with value [42] as a time value: unit is missing or unrecognized"));
}
}
@Test @Test
public void testInternalVersioningInitialDelete() throws Exception { public void testInternalVersioningInitialDelete() throws Exception {
createIndex("test"); createIndex("test");

Some files were not shown because too many files have changed in this diff Show More