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:
commit
6dd6ce2f8f
|
@ -240,7 +240,11 @@ def generate_index(client, version, index_name):
|
|||
client.indices.create(index=index_name, body={
|
||||
'settings': {
|
||||
'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
|
||||
})
|
||||
|
@ -261,7 +265,11 @@ def snapshot_index(client, version, repo_dir):
|
|||
# Add bogus persistent settings to make sure they can be restored
|
||||
client.cluster.put_settings(body={
|
||||
'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={
|
||||
|
|
|
@ -45,7 +45,7 @@
|
|||
type: test
|
||||
id: 1
|
||||
body: { foo: bar }
|
||||
ttl: 100000
|
||||
ttl: 100000ms
|
||||
- do:
|
||||
get:
|
||||
index: test_1
|
||||
|
|
|
@ -40,7 +40,7 @@
|
|||
type: test
|
||||
id: 1
|
||||
body: { foo: bar }
|
||||
ttl: 100000
|
||||
ttl: 100000ms
|
||||
- do:
|
||||
get:
|
||||
index: test_1
|
||||
|
|
|
@ -44,7 +44,7 @@
|
|||
body:
|
||||
doc: { foo: baz }
|
||||
upsert: { foo: bar }
|
||||
ttl: 100000
|
||||
ttl: 100000ms
|
||||
|
||||
- do:
|
||||
get:
|
||||
|
|
|
@ -83,7 +83,7 @@ public class ClusterHealthRequest extends MasterNodeReadRequest<ClusterHealthReq
|
|||
}
|
||||
|
||||
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() {
|
||||
|
|
|
@ -120,7 +120,7 @@ public class DeleteIndexRequest extends MasterNodeRequest<DeleteIndexRequest> im
|
|||
* to <tt>10s</tt>.
|
||||
*/
|
||||
public DeleteIndexRequest timeout(String timeout) {
|
||||
return timeout(TimeValue.parseTimeValue(timeout, null));
|
||||
return timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -317,7 +317,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
|
|||
timestamp = parser.text();
|
||||
} else if ("_ttl".equals(currentFieldName) || "ttl".equals(currentFieldName)) {
|
||||
if (parser.currentToken() == XContentParser.Token.VALUE_STRING) {
|
||||
ttl = TimeValue.parseTimeValue(parser.text(), null).millis();
|
||||
ttl = TimeValue.parseTimeValue(parser.text(), null, currentFieldName).millis();
|
||||
} else {
|
||||
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>.
|
||||
*/
|
||||
public final BulkRequest timeout(String timeout) {
|
||||
return timeout(TimeValue.parseTimeValue(timeout, null));
|
||||
return timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
|
||||
}
|
||||
|
||||
public TimeValue timeout() {
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
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")));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -93,7 +93,7 @@ public class SearchScrollRequest extends ActionRequest<SearchScrollRequest> {
|
|||
* If set, will enable scrolling of the search request for the specified timeout.
|
||||
*/
|
||||
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
|
||||
|
|
|
@ -51,7 +51,7 @@ public abstract class AdapterActionFuture<T, L> extends BaseFuture<T> implements
|
|||
|
||||
@Override
|
||||
public T actionGet(String timeout) {
|
||||
return actionGet(TimeValue.parseTimeValue(timeout, null));
|
||||
return actionGet(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".actionGet.timeout"));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -53,7 +53,7 @@ public abstract class AcknowledgedRequest<T extends MasterNodeRequest> extends M
|
|||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
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() {
|
||||
|
|
|
@ -74,7 +74,7 @@ public abstract class BaseNodesRequest<T extends BaseNodesRequest> extends Actio
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public final T timeout(String timeout) {
|
||||
this.timeout = TimeValue.parseTimeValue(timeout, null);
|
||||
this.timeout = TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout");
|
||||
return (T) this;
|
||||
}
|
||||
|
||||
|
|
|
@ -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>.
|
||||
*/
|
||||
public final T timeout(String timeout) {
|
||||
return timeout(TimeValue.parseTimeValue(timeout, null));
|
||||
return timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
|
||||
}
|
||||
|
||||
public TimeValue timeout() {
|
||||
|
|
|
@ -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>.
|
||||
*/
|
||||
public final T timeout(String timeout) {
|
||||
return timeout(TimeValue.parseTimeValue(timeout, null));
|
||||
return timeout(TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout"));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -259,7 +259,7 @@ public class UpdateHelper extends AbstractComponent {
|
|||
if (fetchedTTL instanceof Number) {
|
||||
ttl = ((Number) fetchedTTL).longValue();
|
||||
} else {
|
||||
ttl = TimeValue.parseTimeValue((String) fetchedTTL, null).millis();
|
||||
ttl = TimeValue.parseTimeValue((String) fetchedTTL, null, "_ttl").millis();
|
||||
}
|
||||
}
|
||||
return ttl;
|
||||
|
|
|
@ -27,24 +27,31 @@ import com.google.common.base.Predicate;
|
|||
import com.google.common.collect.*;
|
||||
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.elasticsearch.cluster.*;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.cluster.*;
|
||||
import org.elasticsearch.cluster.DiffableUtils.KeyedReader;
|
||||
import org.elasticsearch.cluster.block.ClusterBlock;
|
||||
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.Strings;
|
||||
import org.elasticsearch.common.collect.HppcMaps;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.settings.loader.SettingsLoader;
|
||||
import org.elasticsearch.common.xcontent.*;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.indices.IndexClosedException;
|
||||
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.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 org.elasticsearch.common.settings.Settings.*;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData> {
|
||||
|
||||
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() {
|
||||
return this.settings;
|
||||
|
@ -1288,6 +1292,80 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<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 {
|
||||
|
||||
private String uuid;
|
||||
|
|
|
@ -24,7 +24,11 @@ import org.elasticsearch.cluster.routing.HashFunction;
|
|||
import org.elasticsearch.cluster.routing.SimpleHashFunction;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
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
|
||||
|
@ -72,29 +76,30 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
|
|||
|
||||
/**
|
||||
* Checks that the index can be upgraded to the current version of the master node.
|
||||
*
|
||||
* <p/>
|
||||
* If the index does need upgrade it returns the index metadata unchanged, otherwise it returns a modified index metadata. If index cannot be
|
||||
* updated the method throws an exception.
|
||||
* If the index does not need upgrade it returns the index metadata unchanged, otherwise it returns a modified index metadata. If index
|
||||
* cannot be updated the method throws an exception.
|
||||
*/
|
||||
public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData) throws Exception {
|
||||
IndexMetaData newMetaData = indexMetaData;
|
||||
newMetaData = checkSupportedVersion(newMetaData);
|
||||
newMetaData = upgradeLegacyRoutingSettings(newMetaData);
|
||||
// Throws an exception if there are too-old segments:
|
||||
checkSupportedVersion(indexMetaData);
|
||||
IndexMetaData newMetaData = upgradeLegacyRoutingSettings(indexMetaData);
|
||||
newMetaData = addDefaultUnitsIfNeeded(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
|
||||
* 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) {
|
||||
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()
|
||||
+ " and upgraded using the upgrade API.");
|
||||
}
|
||||
return indexMetaData;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -139,4 +144,97 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
|
|||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -101,20 +101,20 @@ public class DiskThresholdDecider extends AllocationDecider {
|
|||
DiskThresholdDecider.this.includeRelocations = newRelocationsSetting;
|
||||
}
|
||||
if (newLowWatermark != null) {
|
||||
if (!validWatermarkSetting(newLowWatermark)) {
|
||||
if (!validWatermarkSetting(newLowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK)) {
|
||||
throw new ElasticsearchParseException("Unable to parse low watermark: [" + newLowWatermark + "]");
|
||||
}
|
||||
logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, 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 (!validWatermarkSetting(newHighWatermark)) {
|
||||
if (!validWatermarkSetting(newHighWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK)) {
|
||||
throw new ElasticsearchParseException("Unable to parse high watermark: [" + newHighWatermark + "]");
|
||||
}
|
||||
logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 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) {
|
||||
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 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 + "]");
|
||||
}
|
||||
if (!validWatermarkSetting(highWatermark)) {
|
||||
if (!validWatermarkSetting(highWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK)) {
|
||||
throw new ElasticsearchParseException("Unable to parse high watermark: [" + highWatermark + "]");
|
||||
}
|
||||
// Watermark is expressed in terms of used data, but we need "free" data watermark
|
||||
this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(lowWatermark);
|
||||
this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(highWatermark);
|
||||
|
||||
this.freeBytesThresholdLow = thresholdBytesFromWatermark(lowWatermark);
|
||||
this.freeBytesThresholdHigh = thresholdBytesFromWatermark(highWatermark);
|
||||
this.freeBytesThresholdLow = thresholdBytesFromWatermark(lowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK);
|
||||
this.freeBytesThresholdHigh = thresholdBytesFromWatermark(highWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK);
|
||||
this.includeRelocations = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true);
|
||||
this.rerouteInterval = settings.getAsTime(CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, TimeValue.timeValueSeconds(60));
|
||||
|
||||
|
@ -569,6 +569,7 @@ public class DiskThresholdDecider extends AllocationDecider {
|
|||
try {
|
||||
return RatioValue.parseRatioValue(watermark).getAsPercent();
|
||||
} 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;
|
||||
}
|
||||
}
|
||||
|
@ -577,11 +578,12 @@ public class DiskThresholdDecider extends AllocationDecider {
|
|||
* Attempts to parse the watermark into a {@link ByteSizeValue}, returning
|
||||
* a ByteSizeValue of 0 bytes if the value cannot be parsed.
|
||||
*/
|
||||
public ByteSizeValue thresholdBytesFromWatermark(String watermark) {
|
||||
public ByteSizeValue thresholdBytesFromWatermark(String watermark, String settingName) {
|
||||
try {
|
||||
return ByteSizeValue.parseBytesSizeValue(watermark);
|
||||
return ByteSizeValue.parseBytesSizeValue(watermark, settingName);
|
||||
} 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,
|
||||
* returning true if valid, false if invalid.
|
||||
*/
|
||||
public boolean validWatermarkSetting(String watermark) {
|
||||
public boolean validWatermarkSetting(String watermark, String settingName) {
|
||||
try {
|
||||
RatioValue.parseRatioValue(watermark);
|
||||
return true;
|
||||
} catch (ElasticsearchParseException e) {
|
||||
try {
|
||||
ByteSizeValue.parseBytesSizeValue(watermark);
|
||||
ByteSizeValue.parseBytesSizeValue(watermark, settingName);
|
||||
return true;
|
||||
} catch (ElasticsearchParseException ex) {
|
||||
return false;
|
||||
|
|
|
@ -44,12 +44,14 @@ public interface Validator {
|
|||
public static final Validator TIME = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException("value must not be null");
|
||||
}
|
||||
try {
|
||||
if (TimeValue.parseTimeValue(value, null) == null) {
|
||||
return "cannot parse value [" + value + "] as time";
|
||||
}
|
||||
// This never returns null:
|
||||
TimeValue.parseTimeValue(value, null, setting);
|
||||
} catch (ElasticsearchParseException ex) {
|
||||
return "cannot parse value [" + value + "] as time";
|
||||
return ex.getMessage();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -59,15 +61,16 @@ public interface Validator {
|
|||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
try {
|
||||
TimeValue timeValue = TimeValue.parseTimeValue(value, null);
|
||||
if (timeValue == null) {
|
||||
return "cannot parse value [" + value + "] as time";
|
||||
if (value == null) {
|
||||
throw new NullPointerException("value must not be null");
|
||||
}
|
||||
TimeValue timeValue = TimeValue.parseTimeValue(value, null, setting);
|
||||
assert timeValue != null;
|
||||
if (timeValue.millis() < 0) {
|
||||
return "cannot parse value [" + value + "] as non negative time";
|
||||
}
|
||||
} catch (ElasticsearchParseException ex) {
|
||||
return "cannot parse value [" + value + "] as time";
|
||||
return ex.getMessage();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -197,7 +200,7 @@ public interface Validator {
|
|||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
try {
|
||||
parseBytesSizeValue(value);
|
||||
parseBytesSizeValue(value, setting);
|
||||
} catch (ElasticsearchParseException ex) {
|
||||
return ex.getMessage();
|
||||
}
|
||||
|
@ -247,7 +250,7 @@ public interface Validator {
|
|||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
try {
|
||||
parseBytesSizeValueOrHeapRatio(value);
|
||||
parseBytesSizeValueOrHeapRatio(value, setting);
|
||||
} catch (ElasticsearchParseException ex) {
|
||||
return ex.getMessage();
|
||||
}
|
||||
|
|
|
@ -25,11 +25,11 @@ package org.elasticsearch.common;
|
|||
public class Booleans {
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the sequence is neither of the following:
|
||||
* <tt>false</tt>, <tt>0</tt>, <tt>off</tt>, <tt>no</tt>,
|
||||
* otherwise <code>false</code>
|
||||
* Returns <code>false</code> if text is in <tt>false</tt>, <tt>0</tt>, <tt>off</tt>, <tt>no</tt>; else, true
|
||||
*/
|
||||
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) {
|
||||
return defaultValue;
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ public class Booleans {
|
|||
* @return true/false
|
||||
* 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);
|
||||
if (isFalse) {
|
||||
|
|
|
@ -56,12 +56,25 @@ import static org.elasticsearch.common.unit.SizeValue.parseSizeValue;
|
|||
import static org.elasticsearch.common.unit.TimeValue.parseTimeValue;
|
||||
|
||||
/**
|
||||
* An immutable {@code Settings} implementation.
|
||||
* An immutable settings implementation.
|
||||
*/
|
||||
public final class Settings implements ToXContent {
|
||||
|
||||
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 final ImmutableMap<String, String> forcedUnderscoreSettings;
|
||||
|
@ -417,7 +430,7 @@ public final class Settings implements ToXContent {
|
|||
* returns the default value provided.
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
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
|
||||
*/
|
||||
public Builder put(String setting, long value, TimeUnit timeUnit) {
|
||||
put(setting, timeUnit.toMillis(value));
|
||||
put(setting, timeUnit.toMillis(value) + "ms");
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -965,7 +999,7 @@ public final class Settings implements ToXContent {
|
|||
* @return The builder
|
||||
*/
|
||||
public Builder put(String setting, long value, ByteSizeUnit sizeUnit) {
|
||||
put(setting, sizeUnit.toBytes(value));
|
||||
put(setting, sizeUnit.toBytes(value) + "b");
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -20,18 +20,18 @@
|
|||
package org.elasticsearch.common.unit;
|
||||
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Locale;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class ByteSizeValue implements Serializable, Streamable {
|
||||
|
||||
private long size;
|
||||
|
@ -171,41 +171,55 @@ public class ByteSizeValue implements Serializable, Streamable {
|
|||
return Strings.format1Decimals(value, suffix);
|
||||
}
|
||||
|
||||
public static ByteSizeValue parseBytesSizeValue(String sValue) throws ElasticsearchParseException {
|
||||
return parseBytesSizeValue(sValue, null);
|
||||
public static ByteSizeValue parseBytesSizeValue(String sValue, String settingName) throws ElasticsearchParseException {
|
||||
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) {
|
||||
return defaultValue;
|
||||
}
|
||||
long bytes;
|
||||
try {
|
||||
String lastTwoChars = sValue.substring(sValue.length() - Math.min(2, sValue.length())).toLowerCase(Locale.ROOT);
|
||||
if (lastTwoChars.endsWith("k")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C1);
|
||||
} else if (lastTwoChars.endsWith("kb")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C1);
|
||||
} else if (lastTwoChars.endsWith("m")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C2);
|
||||
} else if (lastTwoChars.endsWith("mb")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C2);
|
||||
} else if (lastTwoChars.endsWith("g")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C3);
|
||||
} else if (lastTwoChars.endsWith("gb")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C3);
|
||||
} else if (lastTwoChars.endsWith("t")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C4);
|
||||
} else if (lastTwoChars.endsWith("tb")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C4);
|
||||
} else if (lastTwoChars.endsWith("p")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * ByteSizeUnit.C5);
|
||||
} else if (lastTwoChars.endsWith("pb")) {
|
||||
bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C5);
|
||||
} else if (lastTwoChars.endsWith("b")) {
|
||||
bytes = Long.parseLong(sValue.substring(0, sValue.length() - 1));
|
||||
String lowerSValue = sValue.toLowerCase(Locale.ROOT).trim();
|
||||
if (lowerSValue.endsWith("k")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C1);
|
||||
} else if (lowerSValue.endsWith("kb")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C1);
|
||||
} else if (lowerSValue.endsWith("m")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C2);
|
||||
} else if (lowerSValue.endsWith("mb")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C2);
|
||||
} else if (lowerSValue.endsWith("g")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C3);
|
||||
} else if (lowerSValue.endsWith("gb")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C3);
|
||||
} else if (lowerSValue.endsWith("t")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C4);
|
||||
} else if (lowerSValue.endsWith("tb")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C4);
|
||||
} else if (lowerSValue.endsWith("p")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * ByteSizeUnit.C5);
|
||||
} else if (lowerSValue.endsWith("pb")) {
|
||||
bytes = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)) * ByteSizeUnit.C5);
|
||||
} else if (lowerSValue.endsWith("b")) {
|
||||
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 {
|
||||
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) {
|
||||
throw new ElasticsearchParseException("Failed to parse [" + sValue + "]", e);
|
||||
|
|
|
@ -152,7 +152,7 @@ public final class Fuzziness implements ToXContent {
|
|||
if (this == AUTO) {
|
||||
return TimeValue.timeValueMillis(1);
|
||||
} else {
|
||||
return TimeValue.parseTimeValue(fuzziness.toString(), null);
|
||||
return TimeValue.parseTimeValue(fuzziness.toString(), null, "fuzziness");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package org.elasticsearch.common.unit;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
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
|
||||
* <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>. */
|
||||
public static ByteSizeValue parseBytesSizeValueOrHeapRatio(String sValue) {
|
||||
public static ByteSizeValue parseBytesSizeValueOrHeapRatio(String sValue, String settingName) {
|
||||
settingName = Objects.requireNonNull(settingName);
|
||||
if (sValue != null && sValue.endsWith("%")) {
|
||||
final String percentAsString = sValue.substring(0, sValue.length() - 1);
|
||||
try {
|
||||
|
@ -44,7 +47,7 @@ public enum MemorySizeValue {
|
|||
throw new ElasticsearchParseException("Failed to parse [" + percentAsString + "] as a double", e);
|
||||
}
|
||||
} else {
|
||||
return parseBytesSizeValue(sValue);
|
||||
return parseBytesSizeValue(sValue, settingName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,10 +20,12 @@
|
|||
package org.elasticsearch.common.unit;
|
||||
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.joda.time.Period;
|
||||
import org.joda.time.PeriodType;
|
||||
import org.joda.time.format.PeriodFormat;
|
||||
|
@ -31,11 +33,10 @@ import org.joda.time.format.PeriodFormatter;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Locale;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class TimeValue implements Serializable, Streamable {
|
||||
|
||||
/** How many nano-seconds in one milli-second */
|
||||
|
@ -228,28 +229,41 @@ public class TimeValue implements Serializable, Streamable {
|
|||
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) {
|
||||
return defaultValue;
|
||||
}
|
||||
try {
|
||||
long millis;
|
||||
if (sValue.endsWith("S")) {
|
||||
millis = Long.parseLong(sValue.substring(0, sValue.length() - 1));
|
||||
} else if (sValue.endsWith("ms")) {
|
||||
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)));
|
||||
} else if (sValue.endsWith("s")) {
|
||||
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 1000);
|
||||
} else if (sValue.endsWith("m")) {
|
||||
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 60 * 1000);
|
||||
} else if (sValue.endsWith("H") || sValue.endsWith("h")) {
|
||||
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 60 * 60 * 1000);
|
||||
} else if (sValue.endsWith("d")) {
|
||||
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 24 * 60 * 60 * 1000);
|
||||
} else if (sValue.endsWith("w")) {
|
||||
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 7 * 24 * 60 * 60 * 1000);
|
||||
String lowerSValue = sValue.toLowerCase(Locale.ROOT).trim();
|
||||
if (lowerSValue.endsWith("ms")) {
|
||||
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 2)));
|
||||
} else if (lowerSValue.endsWith("s")) {
|
||||
millis = (long) Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 1000;
|
||||
} else if (lowerSValue.endsWith("m")) {
|
||||
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 60 * 1000);
|
||||
} else if (lowerSValue.endsWith("h")) {
|
||||
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 60 * 60 * 1000);
|
||||
} else if (lowerSValue.endsWith("d")) {
|
||||
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.length() - 1)) * 24 * 60 * 60 * 1000);
|
||||
} else if (lowerSValue.endsWith("w")) {
|
||||
millis = (long) (Double.parseDouble(lowerSValue.substring(0, lowerSValue.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 {
|
||||
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);
|
||||
} catch (NumberFormatException e) {
|
||||
|
|
|
@ -377,7 +377,7 @@ public class XContentMapValues {
|
|||
if (node instanceof Number) {
|
||||
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) {
|
||||
|
|
|
@ -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
|
||||
* {@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.
|
||||
*/
|
||||
private void pre20Upgrade() throws Exception {
|
||||
|
|
|
@ -116,7 +116,14 @@ public class MetaStateService extends AbstractComponent {
|
|||
* Loads the global state, *without* index state, see {@link #loadFullState()} for that.
|
||||
*/
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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_GC_DELETES = TimeValue.timeValueSeconds(60);
|
||||
public static final ByteSizeValue DEFAUTL_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB);
|
||||
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb");
|
||||
public static final ByteSizeValue DEFAULT_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB);
|
||||
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb", "INACTIVE_SHARD_INDEXING_BUFFER");
|
||||
|
||||
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.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);
|
||||
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();
|
||||
versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
|
||||
updateVersionMapSize();
|
||||
|
@ -180,7 +180,7 @@ public final class EngineConfig {
|
|||
double percent = Double.parseDouble(versionMapSizeSetting.substring(0, versionMapSizeSetting.length() - 1));
|
||||
versionMapSize = new ByteSizeValue((long) (((double) indexingBufferSize.bytes() * (percent / 100))));
|
||||
} else {
|
||||
versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting);
|
||||
versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting, INDEX_VERSION_MAP_SIZE);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -170,7 +170,7 @@ public class SourceFieldMapper extends AbstractFieldMapper implements RootMapper
|
|||
builder.compressThreshold(((Number) fieldNode).longValue());
|
||||
builder.compress(true);
|
||||
} else {
|
||||
builder.compressThreshold(ByteSizeValue.parseBytesSizeValue(fieldNode.toString()).bytes());
|
||||
builder.compressThreshold(ByteSizeValue.parseBytesSizeValue(fieldNode.toString(), "compress_threshold").bytes());
|
||||
builder.compress(true);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -207,7 +207,7 @@ public class TTLFieldMapper extends LongFieldMapper implements RootMapper {
|
|||
if (context.sourceToParse().ttl() < 0) { // no ttl has been provided externally
|
||||
long ttl;
|
||||
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 {
|
||||
ttl = context.parser().longValue(coerce.value());
|
||||
}
|
||||
|
|
|
@ -269,9 +269,9 @@ public abstract class DecayFunctionParser implements ScoreFunctionParser {
|
|||
if (scaleString == null) {
|
||||
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();
|
||||
val = TimeValue.parseTimeValue(offsetString, TimeValue.timeValueHours(24));
|
||||
val = TimeValue.parseTimeValue(offsetString, TimeValue.timeValueHours(24), getClass().getSimpleName() + ".offset");
|
||||
double offset = val.getMillis();
|
||||
IndexNumericFieldData numericFieldData = parseContext.getForField(dateFieldMapper);
|
||||
return new NumericFieldDataScoreFunction(origin, scale, decay, offset, getDecayFunction(), numericFieldData, mode);
|
||||
|
|
|
@ -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_BUFFER_SIZE = "index.translog.fs.buffer_size";
|
||||
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 BigArrays bigArrays;
|
||||
|
@ -73,7 +73,7 @@ public final class TranslogConfig {
|
|||
this.threadPool = threadPool;
|
||||
this.bigArrays = bigArrays;
|
||||
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));
|
||||
if (syncInterval.millis() > 0 && threadPool != null) {
|
||||
|
|
|
@ -65,7 +65,7 @@ public class IndicesFilterCache extends AbstractComponent implements QueryCache,
|
|||
public IndicesFilterCache(Settings settings) {
|
||||
super(settings);
|
||||
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);
|
||||
logger.debug("using [node] weighted filter cache with size [{}], actual_size [{}], max filter count [{}]",
|
||||
sizeString, size, count);
|
||||
|
|
|
@ -128,7 +128,7 @@ public class IndicesQueryCache extends AbstractComponent implements RemovalListe
|
|||
}
|
||||
|
||||
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()
|
||||
.maximumWeight(sizeInBytes).weigher(new QueryCacheWeigher()).removalListener(this);
|
||||
|
|
|
@ -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 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
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.node.internal;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.UnmodifiableIterator;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.common.Booleans;
|
||||
import org.elasticsearch.common.Names;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.cli.Terminal;
|
||||
|
@ -153,6 +154,10 @@ public class InternalSettingsPreparer {
|
|||
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);
|
||||
environment = new Environment(v1);
|
||||
|
||||
|
|
|
@ -409,7 +409,7 @@ public class PluginManager {
|
|||
case "timeout":
|
||||
case "-timeout":
|
||||
String timeoutValue = getCommandValue(args, ++c, "--timeout");
|
||||
timeout = TimeValue.parseTimeValue(timeoutValue, DEFAULT_TIMEOUT);
|
||||
timeout = TimeValue.parseTimeValue(timeoutValue, DEFAULT_TIMEOUT, command);
|
||||
break;
|
||||
case "-l":
|
||||
case "--list":
|
||||
|
|
|
@ -140,11 +140,11 @@ public abstract class RestRequest extends ContextAndHeaderHolder implements ToXC
|
|||
}
|
||||
|
||||
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) {
|
||||
return parseBytesSizeValue(param(key), defaultValue);
|
||||
return parseBytesSizeValue(param(key), defaultValue, key);
|
||||
}
|
||||
|
||||
public String[] paramAsStringArray(String key, String[] defaultValue) {
|
||||
|
|
|
@ -56,7 +56,7 @@ public class RestNodesHotThreadsAction extends BaseRestHandler {
|
|||
nodesHotThreadsRequest.threads(request.paramAsInt("threads", nodesHotThreadsRequest.threads()));
|
||||
nodesHotThreadsRequest.ignoreIdleThreads(request.paramAsBoolean("ignore_idle_threads", nodesHotThreadsRequest.ignoreIdleThreads()));
|
||||
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()));
|
||||
client.admin().cluster().nodesHotThreads(nodesHotThreadsRequest, new RestResponseListener<NodesHotThreadsResponse>(channel) {
|
||||
@Override
|
||||
|
|
|
@ -113,7 +113,7 @@ public class RestSearchAction extends BaseRestHandler {
|
|||
|
||||
String scroll = request.param("scroll");
|
||||
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")));
|
||||
|
|
|
@ -63,7 +63,7 @@ public class RestSearchScrollAction extends BaseRestHandler {
|
|||
searchScrollRequest.scrollId(scrollId);
|
||||
String scroll = request.param("scroll");
|
||||
if (scroll != null) {
|
||||
searchScrollRequest.scroll(new Scroll(parseTimeValue(scroll, null)));
|
||||
searchScrollRequest.scroll(new Scroll(parseTimeValue(scroll, null, "scroll")));
|
||||
}
|
||||
|
||||
if (RestActions.hasBodyContent(request)) {
|
||||
|
@ -94,7 +94,7 @@ public class RestSearchScrollAction extends BaseRestHandler {
|
|||
} else if ("scroll_id".equals(currentFieldName) && token == XContentParser.Token.VALUE_STRING) {
|
||||
searchScrollRequest.scrollId(parser.text());
|
||||
} 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 {
|
||||
throw new IllegalArgumentException("Unknown parameter [" + currentFieldName + "] in request body or parameter is of the wrong type[" + token + "] ");
|
||||
}
|
||||
|
|
|
@ -63,4 +63,4 @@ public class DateHistogramInterval {
|
|||
public String toString() {
|
||||
return expression;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -189,7 +189,7 @@ public class DateHistogramParser implements Aggregator.Parser {
|
|||
tzRoundingBuilder = TimeZoneRounding.builder(dateTimeUnit);
|
||||
} else {
|
||||
// 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
|
||||
|
@ -217,9 +217,9 @@ public class DateHistogramParser implements Aggregator.Parser {
|
|||
|
||||
private long parseOffset(String offset) throws IOException {
|
||||
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;
|
||||
return TimeValue.parseTimeValue(offset.substring(beginIndex), null).millis();
|
||||
return TimeValue.parseTimeValue(offset.substring(beginIndex), null, getClass().getSimpleName() + ".parseOffset").millis();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -106,7 +106,7 @@ public class DerivativeParser implements PipelineAggregator.Parser {
|
|||
if (dateTimeUnit != null) {
|
||||
xAxisUnits = dateTimeUnit.field().getDurationField().getUnitMillis();
|
||||
} else {
|
||||
TimeValue timeValue = TimeValue.parseTimeValue(units, null);
|
||||
TimeValue timeValue = TimeValue.parseTimeValue(units, null, getClass().getSimpleName() + ".unit");
|
||||
if (timeValue != null) {
|
||||
xAxisUnits = timeValue.getMillis();
|
||||
}
|
||||
|
|
|
@ -317,7 +317,7 @@ public class SearchSourceBuilder extends ToXContentToBytes {
|
|||
* An optional timeout to control how long search is allowed to take.
|
||||
*/
|
||||
public SearchSourceBuilder timeout(String timeout) {
|
||||
this.timeoutInMillis = TimeValue.parseTimeValue(timeout, null).millis();
|
||||
this.timeoutInMillis = TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout").millis();
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ public class TimeoutParseElement implements SearchParseElement {
|
|||
if (token == XContentParser.Token.VALUE_NUMBER) {
|
||||
context.timeoutInMillis(parser.longValue());
|
||||
} else {
|
||||
context.timeoutInMillis(TimeValue.parseTimeValue(parser.text(), null).millis());
|
||||
context.timeoutInMillis(TimeValue.parseTimeValue(parser.text(), null, "timeout").millis());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -157,7 +157,16 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
final SnapshotId snapshotId = new SnapshotId(request.repository(), request.name());
|
||||
final Snapshot snapshot = repository.readSnapshot(snapshotId);
|
||||
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
|
||||
validateSnapshotRestorable(snapshotId, snapshot);
|
||||
|
|
|
@ -39,8 +39,8 @@ public class FsAppendBenchmark {
|
|||
Path path = PathUtils.get("work/test.log");
|
||||
IOUtils.deleteFilesIgnoringExceptions(path);
|
||||
|
||||
int CHUNK = (int) ByteSizeValue.parseBytesSizeValue("1k").bytes();
|
||||
long DATA = ByteSizeValue.parseBytesSizeValue("10gb").bytes();
|
||||
int CHUNK = (int) ByteSizeValue.parseBytesSizeValue("1k", "CHUNK").bytes();
|
||||
long DATA = ByteSizeValue.parseBytesSizeValue("10gb", "DATA").bytes();
|
||||
|
||||
byte[] data = new byte[CHUNK];
|
||||
new Random().nextBytes(data);
|
||||
|
|
|
@ -397,6 +397,6 @@ public class TermsAggregationSearchBenchmark {
|
|||
totalQueryTime += searchResponse.getTookInMillis();
|
||||
}
|
||||
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"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -131,7 +131,8 @@ public class SimpleClusterStateTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
@Test
|
||||
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");
|
||||
int counter = 0;
|
||||
int numberOfFields = 0;
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
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.state.ClusterStateResponse;
|
||||
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.test.ElasticsearchIntegrationTest;
|
||||
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.common.settings.Settings.settingsBuilder;
|
||||
|
@ -69,11 +71,11 @@ public class AckTests extends ElasticsearchIntegrationTest {
|
|||
createIndex("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()) {
|
||||
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() {
|
||||
createIndex("test");
|
||||
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));
|
||||
}
|
||||
|
||||
|
|
|
@ -59,9 +59,9 @@ public class DiskThresholdDeciderUnitTests extends ElasticsearchTestCase {
|
|||
};
|
||||
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.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("0b")));
|
||||
assertThat(decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test")));
|
||||
assertThat(decider.getFreeDiskThresholdLow(), equalTo(15.0d));
|
||||
assertThat(decider.getUsedDiskThresholdLow(), equalTo(85.0d));
|
||||
assertThat(decider.getRerouteInterval().seconds(), equalTo(60L));
|
||||
|
@ -81,11 +81,11 @@ public class DiskThresholdDeciderUnitTests extends ElasticsearchTestCase {
|
|||
applySettings.onRefreshSettings(newSettings);
|
||||
|
||||
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",
|
||||
decider.getFreeDiskThresholdHigh(), equalTo(30.0d));
|
||||
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",
|
||||
decider.getFreeDiskThresholdLow(), equalTo(0.0d));
|
||||
assertThat("reroute interval should be changed to 30 seconds",
|
||||
|
|
|
@ -21,9 +21,13 @@ package org.elasticsearch.cluster.settings;
|
|||
|
||||
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequestBuilder;
|
||||
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.routing.allocation.decider.DisableAllocationDecider;
|
||||
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.indices.store.IndicesStore;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
|
@ -62,7 +66,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
|
|||
String key2 = DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION;
|
||||
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();
|
||||
|
||||
ClusterUpdateSettingsResponse response1 = client().admin().cluster()
|
||||
|
@ -78,7 +82,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
|
|||
assertThat(response1.getPersistentSettings().get(key1), nullValue());
|
||||
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;
|
||||
|
||||
ClusterUpdateSettingsResponse response2 = client().admin().cluster()
|
||||
|
@ -95,7 +99,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
|
|||
assertThat(response2.getPersistentSettings().get(key2), nullValue());
|
||||
|
||||
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()
|
||||
.prepareUpdateSettings()
|
||||
|
@ -180,4 +184,39 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
|
|||
assertThat(response.getPersistentSettings().get(key1), nullValue());
|
||||
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)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,9 +24,6 @@ import org.junit.Test;
|
|||
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class SettingsValidatorTests extends ElasticsearchTestCase {
|
||||
|
||||
@Test
|
||||
|
@ -84,7 +81,7 @@ public class SettingsValidatorTests extends ElasticsearchTestCase {
|
|||
|
||||
assertThat(Validator.PERCENTAGE.validate("", "asdasd"), 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("", "101%"), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "100%"), nullValue());
|
||||
|
@ -92,7 +89,7 @@ public class SettingsValidatorTests extends ElasticsearchTestCase {
|
|||
assertThat(Validator.PERCENTAGE.validate("", "0%"), nullValue());
|
||||
|
||||
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("", "-1%"), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "101%"), notNullValue());
|
||||
|
|
|
@ -70,29 +70,79 @@ public class ByteSizeValueTests extends ElasticsearchTestCase {
|
|||
|
||||
@Test
|
||||
public void testParsing() {
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42pb").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42P").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42PB").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("54tb").toString(), is("54tb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("54T").toString(), is("54tb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("54TB").toString(), is("54tb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("12gb").toString(), is("12gb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("12G").toString(), is("12gb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("12GB").toString(), is("12gb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("12M").toString(), is("12mb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("1b").toString(), is("1b"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("23kb").toString(), is("23kb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("23k").toString(), is("23kb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("23").toString(), is("23b"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42PB", "testParsing").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42 PB", "testParsing").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42pb", "testParsing").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42 pb", "testParsing").toString(), is("42pb"));
|
||||
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42P", "testParsing").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42 P", "testParsing").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42p", "testParsing").toString(), is("42pb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("42 p", "testParsing").toString(), is("42pb"));
|
||||
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("54TB", "testParsing").toString(), is("54tb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("54 TB", "testParsing").toString(), is("54tb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("54tb", "testParsing").toString(), is("54tb"));
|
||||
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)
|
||||
public void testFailOnEmptyParsing() {
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("").toString(), is("23kb"));
|
||||
assertThat(ByteSizeValue.parseBytesSizeValue("", "emptyParsing").toString(), is("23kb"));
|
||||
}
|
||||
|
||||
@Test(expected = ElasticsearchParseException.class)
|
||||
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");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -132,7 +132,7 @@ public class FuzzinessTests extends ElasticsearchTestCase {
|
|||
assertThat(parser.nextToken(), equalTo(XContentParser.Token.FIELD_NAME));
|
||||
assertThat(parser.nextToken(), equalTo(XContentParser.Token.VALUE_STRING));
|
||||
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));
|
||||
}
|
||||
}
|
||||
|
@ -159,7 +159,7 @@ public class FuzzinessTests extends ElasticsearchTestCase {
|
|||
assertThat(Fuzziness.AUTO.asDouble(), equalTo(1d));
|
||||
assertThat(Fuzziness.AUTO.asLong(), equalTo(1l));
|
||||
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")));
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.common.unit;
|
||||
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
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.lessThan;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class TimeValueTests extends ElasticsearchTestCase {
|
||||
|
||||
@Test
|
||||
public void testSimple() {
|
||||
assertThat(TimeUnit.MILLISECONDS.toMillis(10), equalTo(new TimeValue(10, TimeUnit.MILLISECONDS).millis()));
|
||||
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()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToString() {
|
||||
assertThat("10ms", equalTo(new TimeValue(10, 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()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormat() {
|
||||
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"));
|
||||
|
@ -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"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMinusOne() {
|
||||
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 {
|
||||
BytesStreamOutput out = new BytesStreamOutput();
|
||||
value.writeTo(out);
|
||||
|
@ -79,11 +130,24 @@ public class TimeValueTests extends ElasticsearchTestCase {
|
|||
assertThat(inValue, equalTo(value));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerialize() throws Exception {
|
||||
assertEqualityAfterSerialize(new TimeValue(100, TimeUnit.DAYS));
|
||||
assertEqualityAfterSerialize(new TimeValue(-1));
|
||||
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");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
|||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||
import org.elasticsearch.common.breaker.CircuitBreakingException;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
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")) {
|
||||
HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService(
|
||||
Settings.builder()
|
||||
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, size - 1)
|
||||
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, size - 1, ByteSizeUnit.BYTES)
|
||||
.build(),
|
||||
new NodeSettingsService(Settings.EMPTY));
|
||||
BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();
|
||||
|
@ -356,7 +357,7 @@ public class BigArraysTests extends ElasticsearchSingleNodeTest {
|
|||
final long maxSize = randomIntBetween(1 << 10, 1 << 22);
|
||||
HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService(
|
||||
Settings.builder()
|
||||
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, maxSize)
|
||||
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, maxSize, ByteSizeUnit.BYTES)
|
||||
.build(),
|
||||
new NodeSettingsService(Settings.EMPTY));
|
||||
BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();
|
||||
|
|
|
@ -23,6 +23,8 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.test.ElasticsearchSingleNodeTest;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
|
@ -47,7 +49,11 @@ public class InternalEngineSettingsTest extends ElasticsearchSingleNodeTest {
|
|||
final int iters = between(1, 20);
|
||||
for (int i = 0; i < iters; i++) {
|
||||
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();
|
||||
double versionMapPercent = randomIntBetween(0, 100);
|
||||
long versionMapSizeInMB = randomIntBetween(10, 20);
|
||||
|
@ -55,9 +61,10 @@ public class InternalEngineSettingsTest extends ElasticsearchSingleNodeTest {
|
|||
|
||||
Settings build = Settings.builder()
|
||||
.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)
|
||||
.build();
|
||||
assertEquals(gcDeletes, build.getAsTime(EngineConfig.INDEX_GC_DELETES_SETTING, null).millis());
|
||||
|
||||
client().admin().indices().prepareUpdateSettings("foo").setSettings(build).get();
|
||||
LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig();
|
||||
|
@ -78,7 +85,7 @@ public class InternalEngineSettingsTest extends ElasticsearchSingleNodeTest {
|
|||
}
|
||||
|
||||
Settings settings = Settings.builder()
|
||||
.put(EngineConfig.INDEX_GC_DELETES_SETTING, 1000)
|
||||
.put(EngineConfig.INDEX_GC_DELETES_SETTING, 1000, TimeUnit.MILLISECONDS)
|
||||
.build();
|
||||
client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get();
|
||||
assertEquals(engine.getGcDeletesInMillis(), 1000);
|
||||
|
@ -94,7 +101,7 @@ public class InternalEngineSettingsTest extends ElasticsearchSingleNodeTest {
|
|||
assertTrue(engine.config().isEnableGcDeletes());
|
||||
|
||||
settings = Settings.builder()
|
||||
.put(EngineConfig.INDEX_GC_DELETES_SETTING, 1000)
|
||||
.put(EngineConfig.INDEX_GC_DELETES_SETTING, 1000, TimeUnit.MILLISECONDS)
|
||||
.build();
|
||||
client().admin().indices().prepareUpdateSettings("foo").setSettings(settings).get();
|
||||
assertEquals(engine.getGcDeletesInMillis(), 1000);
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.elasticsearch.common.io.PathUtils;
|
|||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.gateway.GatewayAllocator;
|
||||
import org.elasticsearch.index.merge.policy.MergePolicyModule;
|
||||
|
@ -489,7 +490,7 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
|
|||
.setType("fs").setSettings(settingsBuilder()
|
||||
.put("location", randomRepoPath().toAbsolutePath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
logger.info("--> snapshot");
|
||||
CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test").get();
|
||||
assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.PARTIAL));
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.index.translog;
|
||||
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -34,7 +35,7 @@ public class BufferedTranslogTests extends TranslogTests {
|
|||
protected Translog create(Path path) throws IOException {
|
||||
Settings build = Settings.settingsBuilder()
|
||||
.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();
|
||||
TranslogConfig translogConfig = new TranslogConfig(shardId, path, build, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
|
||||
return new Translog(translogConfig);
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
|
|||
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.discovery.DiscoveryService;
|
||||
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.Type;
|
||||
import org.elasticsearch.snapshots.SnapshotState;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.test.store.MockFSDirectoryService;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
|
@ -126,8 +127,8 @@ public class IndexRecoveryTests extends ElasticsearchIntegrationTest {
|
|||
assertTrue(client().admin().cluster().prepareUpdateSettings()
|
||||
.setTransientSettings(Settings.builder()
|
||||
// one chunk per sec..
|
||||
.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize)
|
||||
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize)
|
||||
.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize, ByteSizeUnit.BYTES)
|
||||
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize, ByteSizeUnit.BYTES)
|
||||
)
|
||||
.get().isAcknowledged());
|
||||
}
|
||||
|
|
|
@ -56,7 +56,8 @@ public class DirectBufferNetworkTests extends ElasticsearchIntegrationTest {
|
|||
public void verifySaneDirectBufferAllocations() throws Exception {
|
||||
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];
|
||||
getRandom().nextBytes(data);
|
||||
|
||||
|
|
|
@ -59,7 +59,7 @@ public class TTLPercolatorTests extends ElasticsearchIntegrationTest {
|
|||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
return settingsBuilder()
|
||||
.put(super.nodeSettings(nodeOrdinal))
|
||||
.put("indices.ttl.interval", PURGE_INTERVAL)
|
||||
.put("indices.ttl.interval", PURGE_INTERVAL, TimeUnit.MILLISECONDS)
|
||||
.build();
|
||||
}
|
||||
|
||||
|
@ -163,7 +163,7 @@ public class TTLPercolatorTests extends ElasticsearchIntegrationTest {
|
|||
public void testEnsureTTLDoesNotCreateIndex() throws IOException, InterruptedException {
|
||||
ensureGreen();
|
||||
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder()
|
||||
.put("indices.ttl.interval", 60) // 60 sec
|
||||
.put("indices.ttl.interval", 60, TimeUnit.SECONDS) // 60 sec
|
||||
.build()).get();
|
||||
|
||||
String typeMapping = XContentFactory.jsonBuilder().startObject().startObject("type1")
|
||||
|
@ -176,7 +176,7 @@ public class TTLPercolatorTests extends ElasticsearchIntegrationTest {
|
|||
.execute().actionGet();
|
||||
ensureGreen();
|
||||
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder()
|
||||
.put("indices.ttl.interval", 1) // 60 sec
|
||||
.put("indices.ttl.interval", 1, TimeUnit.SECONDS)
|
||||
.build()).get();
|
||||
|
||||
for (int i = 0; i < 100; i++) {
|
||||
|
|
|
@ -19,10 +19,13 @@
|
|||
package org.elasticsearch.recovery;
|
||||
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
import org.elasticsearch.test.ElasticsearchSingleNodeTest;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
|
||||
|
||||
@Override
|
||||
|
@ -32,7 +35,7 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
|
|||
|
||||
@Test
|
||||
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
|
||||
public void validate(RecoverySettings recoverySettings, int expectedValue) {
|
||||
assertEquals(expectedValue, recoverySettings.fileChunkSize().bytesAsInt());
|
||||
|
@ -44,7 +47,7 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
|
|||
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
|
||||
public void validate(RecoverySettings recoverySettings, int expectedValue) {
|
||||
assertEquals(expectedValue, recoverySettings.translogSize().bytesAsInt());
|
||||
|
@ -68,31 +71,31 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
|
|||
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
|
||||
public void validate(RecoverySettings recoverySettings, int expectedValue) {
|
||||
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
|
||||
public void validate(RecoverySettings recoverySettings, int expectedValue) {
|
||||
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
|
||||
public void validate(RecoverySettings recoverySettings, int expectedValue) {
|
||||
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
|
||||
public void validate(RecoverySettings recoverySettings, int expectedValue) {
|
||||
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
|
||||
public void validate(RecoverySettings recoverySettings, int expectedValue) {
|
||||
assertEquals(expectedValue, recoverySettings.internalActionLongTimeout().millis());
|
||||
|
@ -120,6 +123,16 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
|
|||
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) {
|
||||
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(key, newValue)).get();
|
||||
validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDeci
|
|||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.discovery.DiscoveryService;
|
||||
import org.elasticsearch.env.NodeEnvironment;
|
||||
|
@ -56,8 +57,8 @@ import org.elasticsearch.indices.recovery.RecoveryTarget;
|
|||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.test.BackgroundIndexer;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.elasticsearch.transport.*;
|
||||
|
@ -397,8 +398,8 @@ public class RelocationTests extends ElasticsearchIntegrationTest {
|
|||
assertTrue(client().admin().cluster().prepareUpdateSettings()
|
||||
.setTransientSettings(Settings.builder()
|
||||
// one chunk per sec..
|
||||
.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize)
|
||||
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize)
|
||||
.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize, ByteSizeUnit.BYTES)
|
||||
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize, ByteSizeUnit.BYTES)
|
||||
)
|
||||
.get().isAcknowledged());
|
||||
|
||||
|
|
|
@ -1304,14 +1304,14 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
|
|||
/**
|
||||
* see issue #9634, negative interval in date_histogram should raise exception
|
||||
*/
|
||||
public void testExeptionOnNegativerInterval() {
|
||||
public void testExceptionOnNegativeInterval() {
|
||||
try {
|
||||
client().prepareSearch("idx")
|
||||
.addAggregation(dateHistogram("histo").field("date").interval(-TimeUnit.DAYS.toMillis(1)).minDocCount(0)).execute()
|
||||
.actionGet();
|
||||
fail();
|
||||
} catch (SearchPhaseExecutionException e) {
|
||||
assertThat(e.toString(), containsString("IllegalArgumentException"));
|
||||
assertThat(e.toString(), containsString("ElasticsearchParseException"));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -47,7 +47,7 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
|||
import static org.hamcrest.core.IsNull.notNullValue;
|
||||
|
||||
@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 LONG_FIELD_NAME = "l_value";
|
||||
|
|
|
@ -211,8 +211,8 @@ public class DerivativeTests extends ElasticsearchIntegrationTest {
|
|||
.prepareSearch("idx")
|
||||
.addAggregation(
|
||||
histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0)
|
||||
.subAggregation(derivative("deriv").setBucketsPaths("_count").unit("1"))
|
||||
.subAggregation(derivative("2nd_deriv").setBucketsPaths("deriv").unit("10"))).execute().actionGet();
|
||||
.subAggregation(derivative("deriv").setBucketsPaths("_count").unit("1ms"))
|
||||
.subAggregation(derivative("2nd_deriv").setBucketsPaths("deriv").unit("10ms"))).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
||||
|
|
|
@ -499,7 +499,7 @@ public class SearchScrollTests extends ElasticsearchIntegrationTest {
|
|||
RestSearchScrollAction.buildFromContent(content, searchScrollRequest);
|
||||
|
||||
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
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.carrotsearch.hppc.IntSet;
|
|||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.action.ListenableActionFuture;
|
||||
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.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.AbstractDiffable;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
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.SnapshotMetaData;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
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.Priority;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -706,7 +708,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
int initialIndices = between(1, 3);
|
||||
logger.info("--> create {} indices", initialIndices);
|
||||
|
@ -813,7 +815,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.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))
|
||||
.put("number_of_replicas", 0)));
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.elasticsearch.cluster.metadata.RepositoriesMetaData;
|
|||
import org.elasticsearch.cluster.metadata.RepositoryMetaData;
|
||||
import org.elasticsearch.common.io.FileSystemUtils;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.repositories.RepositoryException;
|
||||
import org.elasticsearch.repositories.RepositoryVerificationException;
|
||||
import org.elasticsearch.snapshots.mockstore.MockRepositoryModule;
|
||||
|
@ -155,7 +156,7 @@ public class RepositoriesTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(5, 100))
|
||||
.put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES)
|
||||
)
|
||||
.setTimeout("0s").get();
|
||||
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(false));
|
||||
|
@ -165,7 +166,7 @@ public class RepositoriesTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(5, 100))
|
||||
.put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES)
|
||||
).get();
|
||||
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));
|
||||
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.elasticsearch.cluster.service.PendingClusterTask;
|
|||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.store.IndexStore;
|
||||
|
@ -91,7 +92,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.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");
|
||||
ensureGreen();
|
||||
|
@ -232,7 +233,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
createIndex("test");
|
||||
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()
|
||||
.put("location", randomRepoPath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
logger.info("--> create index with foo type");
|
||||
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");
|
||||
|
||||
|
@ -300,7 +301,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
logger.info("--> delete the index and recreate it with bar type");
|
||||
cluster().wipeIndices("test-idx");
|
||||
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"));
|
||||
ensureGreen();
|
||||
|
||||
|
@ -318,7 +319,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
|
||||
logger.info("--> assert that old settings are restored");
|
||||
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
|
||||
|
@ -761,7 +762,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", repo)
|
||||
.put("compress", false)
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
createIndex("test-idx");
|
||||
ensureGreen();
|
||||
|
@ -818,7 +819,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", repo)
|
||||
.put("compress", false)
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
createIndex("test-idx-1", "test-idx-2");
|
||||
ensureYellow();
|
||||
|
@ -857,7 +858,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", repo)
|
||||
.put("compress", false)
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
createIndex("test-idx-1", "test-idx-2");
|
||||
ensureYellow();
|
||||
|
@ -892,7 +893,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", repo)
|
||||
.put("compress", false)
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
createIndex("test-idx-1", "test-idx-2");
|
||||
ensureYellow();
|
||||
|
@ -1247,7 +1248,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", repositoryLocation)
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
createIndex("test-idx");
|
||||
ensureGreen();
|
||||
|
@ -1307,7 +1308,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", repositoryLocation)
|
||||
.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_snapshot_bytes_per_sec", throttleSnapshot ? "0.5k" : "0")));
|
||||
|
||||
|
@ -1463,7 +1464,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.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
|
||||
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
|
||||
assertAcked(client.admin().indices().prepareUpdateSettings("test-idx").setSettings(Settings.builder()
|
||||
.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");
|
||||
|
@ -1511,7 +1512,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
// only one shard
|
||||
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()
|
||||
.put("location", randomRepoPath())
|
||||
.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");
|
||||
|
||||
|
@ -1691,7 +1692,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType(MockRepositoryModule.class.getCanonicalName()).setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
|
||||
.put("block_on_init", true)
|
||||
));
|
||||
|
||||
|
@ -1743,7 +1744,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType(MockRepositoryModule.class.getCanonicalName()).setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
|
||||
));
|
||||
|
||||
createIndex("test-idx");
|
||||
|
@ -1823,7 +1824,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath())
|
||||
.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))
|
||||
.put("number_of_replicas", 0)));
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
|
|||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||
import org.elasticsearch.common.io.FileSystemUtils;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.test.ElasticsearchBackwardsCompatIntegrationTest;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -58,7 +59,7 @@ public class SnapshotBackwardsCompatibilityTest extends ElasticsearchBackwardsCo
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", randomRepoPath().toAbsolutePath())
|
||||
.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[] indicesAfter = new String[randomIntBetween(2,5)];
|
||||
for (int i = 0; i < indicesBefore.length; i++) {
|
||||
|
@ -168,7 +169,7 @@ public class SnapshotBackwardsCompatibilityTest extends ElasticsearchBackwardsCo
|
|||
.setType("fs").setSettings(Settings.settingsBuilder()
|
||||
.put("location", tempDir)
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
// only one shard
|
||||
assertAcked(prepareCreate("test").setSettings(Settings.builder()
|
||||
|
|
|
@ -39,7 +39,7 @@ public class GetStressTest {
|
|||
|
||||
final int NUMBER_OF_NODES = 2;
|
||||
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];
|
||||
for (int i = 0; i < nodes.length; i++) {
|
||||
|
@ -93,4 +93,4 @@ public class GetStressTest {
|
|||
System.out.println("test done.");
|
||||
done.set(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -448,7 +448,7 @@ public final class InternalTestCluster extends TestCluster {
|
|||
if (rarely(random)) {
|
||||
builder.put(TranslogConfig.INDEX_TRANSLOG_SYNC_INTERVAL, 0); // 0 has special meaning to sync each op
|
||||
} 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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -490,7 +490,7 @@ public abstract class AbstractSimpleTransportTests extends ElasticsearchTestCase
|
|||
serviceA.registerRequestHandler("sayHelloTimeoutDelayedResponse", StringMessageRequest.class, ThreadPool.Names.GENERIC, new TransportRequestHandler<StringMessageRequest>() {
|
||||
@Override
|
||||
public void messageReceived(StringMessageRequest request, TransportChannel channel) {
|
||||
TimeValue sleep = TimeValue.parseTimeValue(request.message, null);
|
||||
TimeValue sleep = TimeValue.parseTimeValue(request.message, null, "sleep");
|
||||
try {
|
||||
Thread.sleep(sleep.millis());
|
||||
} catch (InterruptedException e) {
|
||||
|
|
|
@ -57,7 +57,7 @@ public class SimpleTTLTests extends ElasticsearchIntegrationTest {
|
|||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
return settingsBuilder()
|
||||
.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.hash.type", "djb")
|
||||
.build();
|
||||
|
|
|
@ -217,6 +217,21 @@ public class SimpleVersioningTests extends ElasticsearchIntegrationTest {
|
|||
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
|
||||
public void testInternalVersioningInitialDelete() throws Exception {
|
||||
createIndex("test");
|
||||
|
|
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue