diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 1c052597e55..48f07b1a2d5 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -517,7 +517,6 @@ - diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java index da97f492f7a..28b62083d42 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java @@ -249,7 +249,7 @@ public class TransportClusterAllocationExplainAction protected void masterOperation(final ClusterAllocationExplainRequest request, final ClusterState state, final ActionListener listener) { final RoutingNodes routingNodes = state.getRoutingNodes(); - final RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, state.nodes(), + final RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, state, clusterInfoService.getClusterInfo(), System.nanoTime()); ShardRouting foundShard = null; diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index 6004d2af94a..cb86c898487 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -54,10 +54,6 @@ import java.util.function.Predicate; */ public class RoutingNodes implements Iterable { - private final MetaData metaData; - - private final ClusterBlocks blocks; - private final RoutingTable routingTable; private final Map nodesToShards = new HashMap<>(); @@ -66,8 +62,6 @@ public class RoutingNodes implements Iterable { private final Map> assignedShards = new HashMap<>(); - private final ImmutableOpenMap customs; - private final boolean readOnly; private int inactivePrimaryCount = 0; @@ -85,10 +79,7 @@ public class RoutingNodes implements Iterable { public RoutingNodes(ClusterState clusterState, boolean readOnly) { this.readOnly = readOnly; - this.metaData = clusterState.metaData(); - this.blocks = clusterState.blocks(); this.routingTable = clusterState.routingTable(); - this.customs = clusterState.customs(); Map> nodesToShards = new HashMap<>(); // fill in the nodeToShards with the "live" nodes @@ -232,28 +223,6 @@ public class RoutingNodes implements Iterable { return routingTable(); } - public MetaData metaData() { - return this.metaData; - } - - public MetaData getMetaData() { - return metaData(); - } - - public ClusterBlocks blocks() { - return this.blocks; - } - - public ClusterBlocks getBlocks() { - return this.blocks; - } - - public ImmutableOpenMap customs() { - return this.customs; - } - - public T custom(String type) { return (T) customs.get(type); } - public UnassignedShards unassigned() { return this.unassignedShards; } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 608477e55fe..e1bbbb7f4ab 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -90,7 +90,7 @@ public class AllocationService extends AbstractComponent { RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards routingNodes.unassigned().shuffle(); - StartedRerouteAllocation allocation = new StartedRerouteAllocation(allocationDeciders, routingNodes, clusterState.nodes(), startedShards, clusterInfoService.getClusterInfo()); + StartedRerouteAllocation allocation = new StartedRerouteAllocation(allocationDeciders, routingNodes, clusterState, startedShards, clusterInfoService.getClusterInfo()); boolean changed = applyStartedShards(routingNodes, startedShards); if (!changed) { return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData()); @@ -216,7 +216,7 @@ public class AllocationService extends AbstractComponent { RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards routingNodes.unassigned().shuffle(); - FailedRerouteAllocation allocation = new FailedRerouteAllocation(allocationDeciders, routingNodes, clusterState.nodes(), failedShards, clusterInfoService.getClusterInfo()); + FailedRerouteAllocation allocation = new FailedRerouteAllocation(allocationDeciders, routingNodes, clusterState, failedShards, clusterInfoService.getClusterInfo()); boolean changed = false; // as failing primaries also fail associated replicas, we fail replicas first here so that their nodes are added to ignore list List orderedFailedShards = new ArrayList<>(failedShards); @@ -266,7 +266,7 @@ public class AllocationService extends AbstractComponent { // we don't shuffle the unassigned shards here, to try and get as close as possible to // a consistent result of the effect the commands have on the routing // this allows systems to dry run the commands, see the resulting cluster state, and act on it - RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState.nodes(), clusterInfoService.getClusterInfo(), currentNanoTime()); + RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState, clusterInfoService.getClusterInfo(), currentNanoTime()); // don't short circuit deciders, we want a full explanation allocation.debugDecision(true); // we ignore disable allocation, because commands are explicit @@ -305,7 +305,7 @@ public class AllocationService extends AbstractComponent { RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards routingNodes.unassigned().shuffle(); - RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState.nodes(), clusterInfoService.getClusterInfo(), currentNanoTime()); + RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState, clusterInfoService.getClusterInfo(), currentNanoTime()); allocation.debugDecision(debug); if (!reroute(allocation)) { return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData()); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedRerouteAllocation.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedRerouteAllocation.java index 7232e15f033..a13862fed26 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedRerouteAllocation.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedRerouteAllocation.java @@ -21,7 +21,7 @@ package org.elasticsearch.cluster.routing.allocation; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterInfo; -import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -57,8 +57,8 @@ public class FailedRerouteAllocation extends RoutingAllocation { private final List failedShards; - public FailedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, DiscoveryNodes nodes, List failedShards, ClusterInfo clusterInfo) { - super(deciders, routingNodes, nodes, clusterInfo, System.nanoTime()); + public FailedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, ClusterState clusterState, List failedShards, ClusterInfo clusterInfo) { + super(deciders, routingNodes, clusterState, clusterInfo, System.nanoTime()); this.failedShards = failedShards; } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java index 536806c0830..60ca3a8d5fd 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java @@ -20,12 +20,14 @@ package org.elasticsearch.cluster.routing.allocation; import org.elasticsearch.cluster.ClusterInfo; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.Decision; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.index.shard.ShardId; import java.util.HashMap; @@ -118,8 +120,12 @@ public class RoutingAllocation { private final RoutingNodes routingNodes; + private final MetaData metaData; + private final DiscoveryNodes nodes; + private final ImmutableOpenMap customs; + private final AllocationExplanation explanation = new AllocationExplanation(); private final ClusterInfo clusterInfo; @@ -139,13 +145,15 @@ public class RoutingAllocation { * Creates a new {@link RoutingAllocation} * @param deciders {@link AllocationDeciders} to used to make decisions for routing allocations * @param routingNodes Routing nodes in the current cluster - * @param nodes TODO: Documentation + * @param clusterState cluster state before rerouting * @param currentNanoTime the nano time to use for all delay allocation calculation (typically {@link System#nanoTime()}) */ - public RoutingAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, DiscoveryNodes nodes, ClusterInfo clusterInfo, long currentNanoTime) { + public RoutingAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, ClusterState clusterState, ClusterInfo clusterInfo, long currentNanoTime) { this.deciders = deciders; this.routingNodes = routingNodes; - this.nodes = nodes; + this.metaData = clusterState.metaData(); + this.nodes = clusterState.nodes(); + this.customs = clusterState.customs(); this.clusterInfo = clusterInfo; this.currentNanoTime = currentNanoTime; } @@ -184,7 +192,7 @@ public class RoutingAllocation { * @return Metadata of routing nodes */ public MetaData metaData() { - return routingNodes.metaData(); + return metaData; } /** @@ -199,6 +207,10 @@ public class RoutingAllocation { return clusterInfo; } + public T custom(String key) { + return (T)customs.get(key); + } + /** * Get explanations of current routing * @return explanation of routing diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StartedRerouteAllocation.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StartedRerouteAllocation.java index 00f3944ae03..e9570edd9c3 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StartedRerouteAllocation.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/StartedRerouteAllocation.java @@ -20,7 +20,7 @@ package org.elasticsearch.cluster.routing.allocation; import org.elasticsearch.cluster.ClusterInfo; -import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -35,8 +35,8 @@ public class StartedRerouteAllocation extends RoutingAllocation { private final List startedShards; - public StartedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, DiscoveryNodes nodes, List startedShards, ClusterInfo clusterInfo) { - super(deciders, routingNodes, nodes, clusterInfo, System.nanoTime()); + public StartedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, ClusterState clusterState, List startedShards, ClusterInfo clusterInfo) { + super(deciders, routingNodes, clusterState, clusterInfo, System.nanoTime()); this.startedShards = startedShards; } @@ -47,4 +47,4 @@ public class StartedRerouteAllocation extends RoutingAllocation { public List startedShards() { return startedShards; } -} \ No newline at end of file +} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index 7fd3a03a23e..b1d78fa44d0 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -225,7 +225,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards this.weight = weight; this.threshold = threshold; this.routingNodes = allocation.routingNodes(); - metaData = routingNodes.metaData(); + this.metaData = allocation.metaData(); avgShardsPerNode = ((float) metaData.getTotalNumberOfShards()) / routingNodes.size(); buildModelFromAssigned(); } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java index 38a2a39fc7c..7dc8eff3ebd 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java @@ -137,7 +137,7 @@ public class EnableAllocationDecider extends AllocationDecider { return allocation.decision(Decision.YES, NAME, "allocation is explicitly ignoring any disabling of relocation"); } - Settings indexSettings = allocation.routingNodes().metaData().getIndexSafe(shardRouting.index()).getSettings(); + Settings indexSettings = allocation.metaData().getIndexSafe(shardRouting.index()).getSettings(); final Rebalance enable; if (INDEX_ROUTING_REBALANCE_ENABLE_SETTING.exists(indexSettings)) { enable = INDEX_ROUTING_REBALANCE_ENABLE_SETTING.get(indexSettings); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDecider.java index eb59c261214..c862865cb16 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDecider.java @@ -105,7 +105,7 @@ public class FilterAllocationDecider extends AllocationDecider { Decision decision = shouldClusterFilter(node, allocation); if (decision != null) return decision; - decision = shouldIndexFilter(allocation.routingNodes().metaData().getIndexSafe(shardRouting.index()), node, allocation); + decision = shouldIndexFilter(allocation.metaData().getIndexSafe(shardRouting.index()), node, allocation); if (decision != null) return decision; return allocation.decision(Decision.YES, NAME, "node passes include/exclude/require filters"); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ShardsLimitAllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ShardsLimitAllocationDecider.java index eb256516353..e1741c1af7f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ShardsLimitAllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ShardsLimitAllocationDecider.java @@ -86,7 +86,7 @@ public class ShardsLimitAllocationDecider extends AllocationDecider { @Override public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { - IndexMetaData indexMd = allocation.routingNodes().metaData().getIndexSafe(shardRouting.index()); + IndexMetaData indexMd = allocation.metaData().getIndexSafe(shardRouting.index()); final int indexShardLimit = INDEX_TOTAL_SHARDS_PER_NODE_SETTING.get(indexMd.getSettings(), settings); // Capture the limit here in case it changes during this method's // execution @@ -125,7 +125,7 @@ public class ShardsLimitAllocationDecider extends AllocationDecider { @Override public Decision canRemain(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { - IndexMetaData indexMd = allocation.routingNodes().metaData().getIndexSafe(shardRouting.index()); + IndexMetaData indexMd = allocation.metaData().getIndexSafe(shardRouting.index()); final int indexShardLimit = INDEX_TOTAL_SHARDS_PER_NODE_SETTING.get(indexMd.getSettings(), settings); // Capture the limit here in case it changes during this method's // execution diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java index 54cfb6407da..e25a4e690da 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java @@ -98,7 +98,7 @@ public class SnapshotInProgressAllocationDecider extends AllocationDecider { if (!enableRelocation && shardRouting.primary()) { // Only primary shards are snapshotted - SnapshotsInProgress snapshotsInProgress = allocation.routingNodes().custom(SnapshotsInProgress.TYPE); + SnapshotsInProgress snapshotsInProgress = allocation.custom(SnapshotsInProgress.TYPE); if (snapshotsInProgress == null) { // Snapshots are not running return allocation.decision(Decision.YES, NAME, "no snapshots are currently running"); diff --git a/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java b/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java index 8cfe446a29b..73f4b9b2ef0 100644 --- a/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java +++ b/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java @@ -84,7 +84,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent { public boolean allocateUnassigned(RoutingAllocation allocation) { boolean changed = false; final RoutingNodes routingNodes = allocation.routingNodes(); - final MetaData metaData = routingNodes.metaData(); + final MetaData metaData = allocation.metaData(); final RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator = routingNodes.unassigned().iterator(); while (unassignedIterator.hasNext()) { diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/SortProcessor.java b/core/src/main/java/org/elasticsearch/ingest/processor/SortProcessor.java new file mode 100644 index 00000000000..313a8bed7fc --- /dev/null +++ b/core/src/main/java/org/elasticsearch/ingest/processor/SortProcessor.java @@ -0,0 +1,137 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.ingest.core.AbstractProcessor; +import org.elasticsearch.ingest.core.AbstractProcessorFactory; +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.core.ConfigurationUtils; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Processor that sorts an array of items. + * Throws exception is the specified field is not an array. + */ +public final class SortProcessor extends AbstractProcessor { + + public static final String TYPE = "sort"; + public static final String FIELD = "field"; + public static final String ORDER = "order"; + public static final String DEFAULT_ORDER = "asc"; + + public enum SortOrder { + ASCENDING("asc"), DESCENDING("desc"); + + private final String direction; + + SortOrder(String direction) { + this.direction = direction; + } + + public String toString() { + return this.direction; + } + + public static SortOrder fromString(String value) { + if (value == null) { + throw new IllegalArgumentException("Sort direction cannot be null"); + } + + if (value.equals(ASCENDING.toString())) { + return ASCENDING; + } else if (value.equals(DESCENDING.toString())) { + return DESCENDING; + } + throw new IllegalArgumentException("Sort direction [" + value + "] not recognized." + + " Valid values are: [asc, desc]"); + } + } + + private final String field; + private final SortOrder order; + + SortProcessor(String tag, String field, SortOrder order) { + super(tag); + this.field = field; + this.order = order; + } + + String getField() { + return field; + } + + SortOrder getOrder() { + return order; + } + + @Override + @SuppressWarnings("unchecked") + public void execute(IngestDocument document) { + List list = document.getFieldValue(field, List.class); + + if (list == null) { + throw new IllegalArgumentException("field [" + field + "] is null, cannot sort."); + } + + if (list.size() <= 1) { + return; + } + + if (order.equals(SortOrder.ASCENDING)) { + Collections.sort(list); + } else { + Collections.sort(list, Collections.reverseOrder()); + } + + document.setFieldValue(field, list); + } + + @Override + public String getType() { + return TYPE; + } + + public final static class Factory extends AbstractProcessorFactory { + + @Override + public SortProcessor doCreate(String processorTag, Map config) throws Exception { + String field = ConfigurationUtils.readStringProperty(TYPE, processorTag, config, FIELD); + try { + SortOrder direction = SortOrder.fromString( + ConfigurationUtils.readStringProperty( + TYPE, + processorTag, + config, + ORDER, + DEFAULT_ORDER)); + return new SortProcessor(processorTag, field, direction); + } catch (IllegalArgumentException e) { + throw ConfigurationUtils.newConfigurationException(TYPE, processorTag, ORDER, e.getMessage()); + } + } + } +} + diff --git a/core/src/main/java/org/elasticsearch/monitor/jvm/JvmGcMonitorService.java b/core/src/main/java/org/elasticsearch/monitor/jvm/JvmGcMonitorService.java index 5a2d591c7dc..01b4c68537e 100644 --- a/core/src/main/java/org/elasticsearch/monitor/jvm/JvmGcMonitorService.java +++ b/core/src/main/java/org/elasticsearch/monitor/jvm/JvmGcMonitorService.java @@ -20,9 +20,11 @@ package org.elasticsearch.monitor.jvm; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.monitor.jvm.JvmStats.GarbageCollector; @@ -30,14 +32,13 @@ import org.elasticsearch.threadpool.ThreadPool; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; import static java.util.Collections.unmodifiableMap; -import static org.elasticsearch.monitor.jvm.JvmStats.jvmStats; -/** - * - */ public class JvmGcMonitorService extends AbstractLifecycleComponent { private final ThreadPool threadPool; @@ -119,12 +120,123 @@ public class JvmGcMonitorService extends AbstractLifecycleComponent[{}]/[{}], all_pools {}"; + @Override protected void doStart() { if (!enabled) { return; } - scheduledFuture = threadPool.scheduleWithFixedDelay(new JvmMonitor(), interval); + scheduledFuture = threadPool.scheduleWithFixedDelay(new JvmMonitor(gcThresholds) { + @Override + void onMonitorFailure(Throwable t) { + logger.debug("failed to monitor", t); + } + + @Override + void onSlowGc(final Threshold threshold, final long seq, final SlowGcEvent slowGcEvent) { + logSlowGc(logger, threshold, seq, slowGcEvent, JvmGcMonitorService::buildPools); + } + }, interval); + } + + static void logSlowGc( + final ESLogger logger, + final JvmMonitor.Threshold threshold, + final long seq, + final JvmMonitor.SlowGcEvent slowGcEvent, + BiFunction pools) { + + final String name = slowGcEvent.currentGc.getName(); + final long elapsed = slowGcEvent.elapsed; + final long totalGcCollectionCount = slowGcEvent.currentGc.getCollectionCount(); + final long currentGcCollectionCount = slowGcEvent.collectionCount; + final TimeValue totalGcCollectionTime = slowGcEvent.currentGc.getCollectionTime(); + final TimeValue currentGcCollectionTime = slowGcEvent.collectionTime; + final JvmStats lastJvmStats = slowGcEvent.lastJvmStats; + final JvmStats currentJvmStats = slowGcEvent.currentJvmStats; + final ByteSizeValue maxHeapUsed = slowGcEvent.maxHeapUsed; + + switch (threshold) { + case WARN: + if (logger.isWarnEnabled()) { + logger.warn( + LOG_MESSAGE, + name, + seq, + totalGcCollectionCount, + currentGcCollectionTime, + currentGcCollectionCount, + TimeValue.timeValueMillis(elapsed), + currentGcCollectionTime, + totalGcCollectionTime, + lastJvmStats.getMem().getHeapUsed(), + currentJvmStats.getMem().getHeapUsed(), + maxHeapUsed, + pools.apply(lastJvmStats, currentJvmStats)); + } + break; + case INFO: + if (logger.isInfoEnabled()) { + logger.info( + LOG_MESSAGE, + name, + seq, + totalGcCollectionCount, + currentGcCollectionTime, + currentGcCollectionCount, + TimeValue.timeValueMillis(elapsed), + currentGcCollectionTime, + totalGcCollectionTime, + lastJvmStats.getMem().getHeapUsed(), + currentJvmStats.getMem().getHeapUsed(), + maxHeapUsed, + pools.apply(lastJvmStats, currentJvmStats)); + } + break; + case DEBUG: + if (logger.isDebugEnabled()) { + logger.debug( + LOG_MESSAGE, + name, + seq, + totalGcCollectionCount, + currentGcCollectionTime, + currentGcCollectionCount, + TimeValue.timeValueMillis(elapsed), + currentGcCollectionTime, + totalGcCollectionTime, + lastJvmStats.getMem().getHeapUsed(), + currentJvmStats.getMem().getHeapUsed(), + maxHeapUsed, + pools.apply(lastJvmStats, currentJvmStats)); + } + break; + } + } + + static String buildPools(JvmStats last, JvmStats current) { + StringBuilder sb = new StringBuilder(); + for (JvmStats.MemoryPool currentPool : current.getMem()) { + JvmStats.MemoryPool prevPool = null; + for (JvmStats.MemoryPool pool : last.getMem()) { + if (pool.getName().equals(currentPool.getName())) { + prevPool = pool; + break; + } + } + sb.append("{[") + .append(currentPool.getName()) + .append("] [") + .append(prevPool == null ? "?" : prevPool.getUsed()) + .append("]->[") + .append(currentPool.getUsed()) + .append("]/[") + .append(currentPool.getMax()) + .append("]}"); + } + return sb.toString(); } @Override @@ -139,12 +251,46 @@ public class JvmGcMonitorService extends AbstractLifecycleComponent gcThresholds; - public JvmMonitor() { + public JvmMonitor(Map gcThresholds) { + this.gcThresholds = Objects.requireNonNull(gcThresholds); } @Override @@ -152,24 +298,28 @@ public class JvmGcMonitorService extends AbstractLifecycleComponent gcThreshold.warnThreshold) { - logger.warn("[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}", - gc.getName(), seq, gc.getCollectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.getTimestamp() - lastJvmStats.getTimestamp()), TimeValue.timeValueMillis(collectionTime), gc.getCollectionTime(), lastJvmStats.getMem().getHeapUsed(), currentJvmStats.getMem().getHeapUsed(), JvmInfo.jvmInfo().getMem().getHeapMax(), buildPools(lastJvmStats, currentJvmStats)); + threshold = Threshold.WARN; } else if (avgCollectionTime > gcThreshold.infoThreshold) { - logger.info("[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}", - gc.getName(), seq, gc.getCollectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.getTimestamp() - lastJvmStats.getTimestamp()), TimeValue.timeValueMillis(collectionTime), gc.getCollectionTime(), lastJvmStats.getMem().getHeapUsed(), currentJvmStats.getMem().getHeapUsed(), JvmInfo.jvmInfo().getMem().getHeapMax(), buildPools(lastJvmStats, currentJvmStats)); - } else if (avgCollectionTime > gcThreshold.debugThreshold && logger.isDebugEnabled()) { - logger.debug("[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}", - gc.getName(), seq, gc.getCollectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.getTimestamp() - lastJvmStats.getTimestamp()), TimeValue.timeValueMillis(collectionTime), gc.getCollectionTime(), lastJvmStats.getMem().getHeapUsed(), currentJvmStats.getMem().getHeapUsed(), JvmInfo.jvmInfo().getMem().getHeapMax(), buildPools(lastJvmStats, currentJvmStats)); + threshold = Threshold.INFO; + } else if (avgCollectionTime > gcThreshold.debugThreshold) { + threshold = Threshold.DEBUG; + } + if (threshold != null) { + onSlowGc(threshold, seq, new SlowGcEvent( + gc, + collections, + TimeValue.timeValueMillis(collectionTime), + elapsed, + lastJvmStats, + currentJvmStats, + JvmInfo.jvmInfo().getMem().getHeapMax())); } } + lastTime = currentTime; lastJvmStats = currentJvmStats; } - private String buildPools(JvmStats prev, JvmStats current) { - StringBuilder sb = new StringBuilder(); - for (JvmStats.MemoryPool currentPool : current.getMem()) { - JvmStats.MemoryPool prevPool = null; - for (JvmStats.MemoryPool pool : prev.getMem()) { - if (pool.getName().equals(currentPool.getName())) { - prevPool = pool; - break; - } - } - sb.append("{[").append(currentPool.getName()) - .append("] [").append(prevPool == null ? "?" : prevPool.getUsed()).append("]->[").append(currentPool.getUsed()).append("]/[").append(currentPool.getMax()).append("]}"); - } - return sb.toString(); + JvmStats jvmStats() { + return JvmStats.jvmStats(); } + + long now() { + return System.nanoTime(); + } + + abstract void onSlowGc(final Threshold threshold, final long seq, final SlowGcEvent slowGcEvent); + } + } diff --git a/core/src/main/java/org/elasticsearch/node/NodeModule.java b/core/src/main/java/org/elasticsearch/node/NodeModule.java index 8565b14be8e..8c9b3e362fc 100644 --- a/core/src/main/java/org/elasticsearch/node/NodeModule.java +++ b/core/src/main/java/org/elasticsearch/node/NodeModule.java @@ -37,6 +37,7 @@ import org.elasticsearch.ingest.processor.LowercaseProcessor; import org.elasticsearch.ingest.processor.RemoveProcessor; import org.elasticsearch.ingest.processor.RenameProcessor; import org.elasticsearch.ingest.processor.SetProcessor; +import org.elasticsearch.ingest.processor.SortProcessor; import org.elasticsearch.ingest.processor.SplitProcessor; import org.elasticsearch.ingest.processor.TrimProcessor; import org.elasticsearch.ingest.processor.UppercaseProcessor; @@ -78,6 +79,7 @@ public class NodeModule extends AbstractModule { registerProcessor(FailProcessor.TYPE, (templateService, registry) -> new FailProcessor.Factory(templateService)); registerProcessor(ForEachProcessor.TYPE, (templateService, registry) -> new ForEachProcessor.Factory(registry)); registerProcessor(DateIndexNameProcessor.TYPE, (templateService, registry) -> new DateIndexNameProcessor.Factory()); + registerProcessor(SortProcessor.TYPE, (templateService, registry) -> new SortProcessor.Factory()); } @Override diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java index 4953a39d4b5..f3ad47910e4 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java @@ -30,8 +30,8 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.profile.CollectorResult; -import org.elasticsearch.search.profile.InternalProfileCollector; +import org.elasticsearch.search.profile.query.CollectorResult; +import org.elasticsearch.search.profile.query.InternalProfileCollector; import org.elasticsearch.search.query.QueryPhaseExecutionException; import java.io.IOException; @@ -125,7 +125,7 @@ public class AggregationPhase implements SearchPhase { Collections.emptyList()); collector = profileCollector; // start a new profile with this collector - context.getProfilers().addProfiler().setCollector(profileCollector); + context.getProfilers().addQueryProfiler().setCollector(profileCollector); } globalsCollector.preCollection(); context.searcher().search(query, collector); diff --git a/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java b/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java index 471ce26d297..df1007ebc71 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java @@ -33,10 +33,10 @@ import org.apache.lucene.search.Weight; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.search.dfs.AggregatedDfs; -import org.elasticsearch.search.profile.QueryProfileBreakdown; -import org.elasticsearch.search.profile.ProfileWeight; -import org.elasticsearch.search.profile.QueryProfiler; -import org.elasticsearch.search.profile.QueryTimingType; +import org.elasticsearch.search.profile.query.ProfileWeight; +import org.elasticsearch.search.profile.query.QueryProfileBreakdown; +import org.elasticsearch.search.profile.query.QueryProfiler; +import org.elasticsearch.search.profile.query.QueryTimingType; import java.io.IOException; diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileResult.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileResult.java index 165244e8d69..4c745a5127d 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/ProfileResult.java +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileResult.java @@ -43,7 +43,7 @@ import java.util.Map; * Each InternalProfileResult has a List of InternalProfileResults, which will contain * "children" queries if applicable */ -final class ProfileResult implements Writeable, ToXContent { +public final class ProfileResult implements Writeable, ToXContent { private static final ParseField TYPE = new ParseField("type"); private static final ParseField DESCRIPTION = new ParseField("description"); diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java index e35a3c2e2aa..9def3db7582 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.profile.query.CollectorResult; import java.io.IOException; import java.util.ArrayList; diff --git a/core/src/main/java/org/elasticsearch/search/profile/Profilers.java b/core/src/main/java/org/elasticsearch/search/profile/Profilers.java index 38b69d38b15..e9e6d88db18 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/Profilers.java +++ b/core/src/main/java/org/elasticsearch/search/profile/Profilers.java @@ -20,6 +20,7 @@ package org.elasticsearch.search.profile; import org.elasticsearch.search.internal.ContextIndexSearcher; +import org.elasticsearch.search.profile.query.QueryProfiler; import java.util.ArrayList; import java.util.Collections; @@ -29,31 +30,31 @@ import java.util.List; public final class Profilers { private final ContextIndexSearcher searcher; - private final List profilers; + private final List queryProfilers; /** Sole constructor. This {@link Profilers} instance will initially wrap one {@link QueryProfiler}. */ public Profilers(ContextIndexSearcher searcher) { this.searcher = searcher; - this.profilers = new ArrayList<>(); - addProfiler(); + this.queryProfilers = new ArrayList<>(); + addQueryProfiler(); } /** Switch to a new profile. */ - public QueryProfiler addProfiler() { + public QueryProfiler addQueryProfiler() { QueryProfiler profiler = new QueryProfiler(); searcher.setProfiler(profiler); - profilers.add(profiler); + queryProfilers.add(profiler); return profiler; } /** Get the current profiler. */ - public QueryProfiler getCurrent() { - return profilers.get(profilers.size() - 1); + public QueryProfiler getCurrentQueryProfiler() { + return queryProfilers.get(queryProfilers.size() - 1); } /** Return the list of all created {@link QueryProfiler}s so far. */ - public List getProfilers() { - return Collections.unmodifiableList(profilers); + public List getQueryProfilers() { + return Collections.unmodifiableList(queryProfilers); } } diff --git a/core/src/main/java/org/elasticsearch/search/profile/SearchProfileShardResults.java b/core/src/main/java/org/elasticsearch/search/profile/SearchProfileShardResults.java index a38c0813dc5..bf265dd9a7e 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/SearchProfileShardResults.java +++ b/core/src/main/java/org/elasticsearch/search/profile/SearchProfileShardResults.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.profile.query.QueryProfiler; import java.io.IOException; import java.util.ArrayList; diff --git a/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java b/core/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java similarity index 99% rename from core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java rename to core/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java index 32b9ccfa942..c517c8730e4 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; diff --git a/core/src/main/java/org/elasticsearch/search/profile/InternalProfileCollector.java b/core/src/main/java/org/elasticsearch/search/profile/query/InternalProfileCollector.java similarity index 98% rename from core/src/main/java/org/elasticsearch/search/profile/InternalProfileCollector.java rename to core/src/main/java/org/elasticsearch/search/profile/query/InternalProfileCollector.java index 48d165e3095..e892abaab22 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/InternalProfileCollector.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/InternalProfileCollector.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Collector; diff --git a/core/src/main/java/org/elasticsearch/search/profile/InternalQueryProfileTree.java b/core/src/main/java/org/elasticsearch/search/profile/query/InternalQueryProfileTree.java similarity index 98% rename from core/src/main/java/org/elasticsearch/search/profile/InternalQueryProfileTree.java rename to core/src/main/java/org/elasticsearch/search/profile/query/InternalQueryProfileTree.java index 9ebee2f1635..5b92ef8b2a9 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/InternalQueryProfileTree.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/InternalQueryProfileTree.java @@ -17,9 +17,10 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.search.Query; +import org.elasticsearch.search.profile.ProfileResult; import java.util.ArrayList; import java.util.Collections; diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileCollector.java b/core/src/main/java/org/elasticsearch/search/profile/query/ProfileCollector.java similarity index 98% rename from core/src/main/java/org/elasticsearch/search/profile/ProfileCollector.java rename to core/src/main/java/org/elasticsearch/search/profile/query/ProfileCollector.java index 7d7538c9117..c65d0978906 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/ProfileCollector.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/ProfileCollector.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Collector; diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileScorer.java b/core/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java similarity index 99% rename from core/src/main/java/org/elasticsearch/search/profile/ProfileScorer.java rename to core/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java index 939ac85c871..51d0b14fc96 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/ProfileScorer.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Scorer; diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileWeight.java b/core/src/main/java/org/elasticsearch/search/profile/query/ProfileWeight.java similarity index 98% rename from core/src/main/java/org/elasticsearch/search/profile/ProfileWeight.java rename to core/src/main/java/org/elasticsearch/search/profile/query/ProfileWeight.java index 9546f691616..9ca33f84542 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/ProfileWeight.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/ProfileWeight.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; diff --git a/core/src/main/java/org/elasticsearch/search/profile/QueryProfileBreakdown.java b/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfileBreakdown.java similarity index 91% rename from core/src/main/java/org/elasticsearch/search/profile/QueryProfileBreakdown.java rename to core/src/main/java/org/elasticsearch/search/profile/query/QueryProfileBreakdown.java index fdb13a49977..d0608eb01af 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/QueryProfileBreakdown.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfileBreakdown.java @@ -17,7 +17,9 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; + +import org.elasticsearch.search.profile.AbstractProfileBreakdown; /** * A record of timings for the various operations that may happen during query execution. diff --git a/core/src/main/java/org/elasticsearch/search/profile/QueryProfiler.java b/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfiler.java similarity index 97% rename from core/src/main/java/org/elasticsearch/search/profile/QueryProfiler.java rename to core/src/main/java/org/elasticsearch/search/profile/query/QueryProfiler.java index f8fcf5fbf13..57341ee132f 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/QueryProfiler.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/QueryProfiler.java @@ -17,9 +17,10 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.search.Query; +import org.elasticsearch.search.profile.ProfileResult; import java.util.List; import java.util.Objects; diff --git a/core/src/main/java/org/elasticsearch/search/profile/QueryTimingType.java b/core/src/main/java/org/elasticsearch/search/profile/query/QueryTimingType.java similarity index 95% rename from core/src/main/java/org/elasticsearch/search/profile/QueryTimingType.java rename to core/src/main/java/org/elasticsearch/search/profile/query/QueryTimingType.java index 5fb3b29dee3..5f194a7d5f1 100644 --- a/core/src/main/java/org/elasticsearch/search/profile/QueryTimingType.java +++ b/core/src/main/java/org/elasticsearch/search/profile/query/QueryTimingType.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import java.util.Locale; diff --git a/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java index da849828e7a..df68064f617 100644 --- a/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -52,10 +52,10 @@ import org.elasticsearch.search.SearchService; import org.elasticsearch.search.aggregations.AggregationPhase; import org.elasticsearch.search.internal.ScrollContext; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.profile.CollectorResult; -import org.elasticsearch.search.profile.InternalProfileCollector; import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResults; +import org.elasticsearch.search.profile.query.CollectorResult; +import org.elasticsearch.search.profile.query.InternalProfileCollector; import org.elasticsearch.search.rescore.RescorePhase; import org.elasticsearch.search.rescore.RescoreSearchContext; import org.elasticsearch.search.sort.SortAndFormats; @@ -113,7 +113,7 @@ public class QueryPhase implements SearchPhase { if (searchContext.getProfilers() != null) { List shardResults = SearchProfileShardResults - .buildShardResults(searchContext.getProfilers().getProfilers()); + .buildShardResults(searchContext.getProfilers().getQueryProfilers()); searchContext.queryResult().profileResults(shardResults); } } @@ -365,7 +365,7 @@ public class QueryPhase implements SearchPhase { try { if (collector != null) { if (doProfile) { - searchContext.getProfilers().getCurrent().setCollector((InternalProfileCollector) collector); + searchContext.getProfilers().getCurrentQueryProfiler().setCollector((InternalProfileCollector) collector); } searcher.search(query, collector); } @@ -386,7 +386,7 @@ public class QueryPhase implements SearchPhase { if (searchContext.getProfilers() != null) { List shardResults = SearchProfileShardResults - .buildShardResults(searchContext.getProfilers().getProfilers()); + .buildShardResults(searchContext.getProfilers().getQueryProfilers()); searchContext.queryResult().profileResults(shardResults); } diff --git a/core/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java b/core/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java index 8d13f04240e..f47134a45b8 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java @@ -96,7 +96,7 @@ public class AckClusterUpdateSettingsIT extends ESIntegTestCase { for (Client client : clients()) { ClusterState clusterState = getLocalClusterState(client); - assertThat(clusterState.getRoutingNodes().metaData().transientSettings().get("cluster.routing.allocation.exclude._id"), equalTo(excludedNodeId)); + assertThat(clusterState.metaData().transientSettings().get("cluster.routing.allocation.exclude._id"), equalTo(excludedNodeId)); for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { for (IndexShardRoutingTable indexShardRoutingTable : indexRoutingTable) { for (ShardRouting shardRouting : indexShardRoutingTable) { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index a58c9c3f58b..2c4e86ad4b1 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -875,7 +875,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { ) ); ClusterState clusterState = ClusterState.builder(baseClusterState).routingTable(builder.build()).build(); - RoutingAllocation routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), discoveryNodes, clusterInfo, + RoutingAllocation routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), clusterState, clusterInfo, System.nanoTime()); Decision decision = diskThresholdDecider.canRemain(firstRouting, firstRoutingNode, routingAllocation); assertThat(decision.type(), equalTo(Decision.Type.NO)); @@ -896,7 +896,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { ) ); clusterState = ClusterState.builder(baseClusterState).routingTable(builder.build()).build(); - routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), discoveryNodes, clusterInfo, System.nanoTime()); + routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), clusterState, clusterInfo, System.nanoTime()); decision = diskThresholdDecider.canRemain(firstRouting, firstRoutingNode, routingAllocation); assertThat(decision.type(), equalTo(Decision.Type.YES)); @@ -991,7 +991,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { ) ); ClusterState clusterState = ClusterState.builder(baseClusterState).routingTable(builder.build()).build(); - RoutingAllocation routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), discoveryNodes, clusterInfo, + RoutingAllocation routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), clusterState, clusterInfo, System.nanoTime()); Decision decision = diskThresholdDecider.canRemain(firstRouting, firstRoutingNode, routingAllocation); @@ -1051,7 +1051,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { ); clusterState = ClusterState.builder(updateClusterState).routingTable(builder.build()).build(); - routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), discoveryNodes, clusterInfo, System.nanoTime()); + routingAllocation = new RoutingAllocation(null, new RoutingNodes(clusterState), clusterState, clusterInfo, System.nanoTime()); decision = diskThresholdDecider.canRemain(firstRouting, firstRoutingNode, routingAllocation); assertThat(decision.type(), equalTo(Decision.Type.YES)); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java index 3b1799e4b06..d9e9fb95445 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java @@ -136,7 +136,7 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase { ImmutableOpenMap.Builder shardSizes = ImmutableOpenMap.builder(); shardSizes.put("[test][0][p]", 10L); // 10 bytes final ClusterInfo clusterInfo = new ClusterInfo(leastAvailableUsages.build(), mostAvailableUsage.build(), shardSizes.build(), ImmutableOpenMap.of()); - RoutingAllocation allocation = new RoutingAllocation(new AllocationDeciders(Settings.EMPTY, new AllocationDecider[]{decider}), clusterState.getRoutingNodes(), clusterState.nodes(), clusterInfo, System.nanoTime()); + RoutingAllocation allocation = new RoutingAllocation(new AllocationDeciders(Settings.EMPTY, new AllocationDecider[]{decider}), clusterState.getRoutingNodes(), clusterState, clusterInfo, System.nanoTime()); assertEquals(mostAvailableUsage.toString(), Decision.YES, decider.canAllocate(test_0, new RoutingNode("node_0", node_0), allocation)); assertEquals(mostAvailableUsage.toString(), Decision.NO, decider.canAllocate(test_0, new RoutingNode("node_1", node_1), allocation)); } @@ -204,7 +204,7 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase { shardSizes.put("[test][2][p]", 10L); final ClusterInfo clusterInfo = new ClusterInfo(leastAvailableUsages.build(), mostAvailableUsage.build(), shardSizes.build(), shardRoutingMap.build()); - RoutingAllocation allocation = new RoutingAllocation(new AllocationDeciders(Settings.EMPTY, new AllocationDecider[]{decider}), clusterState.getRoutingNodes(), clusterState.nodes(), clusterInfo, System.nanoTime()); + RoutingAllocation allocation = new RoutingAllocation(new AllocationDeciders(Settings.EMPTY, new AllocationDecider[]{decider}), clusterState.getRoutingNodes(), clusterState, clusterInfo, System.nanoTime()); assertEquals(Decision.YES, decider.canRemain(test_0, new RoutingNode("node_0", node_0), allocation)); assertEquals(Decision.NO, decider.canRemain(test_1, new RoutingNode("node_1", node_1), allocation)); try { diff --git a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java index b62d2fe3342..01e717e6fa9 100644 --- a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java @@ -346,7 +346,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { .metaData(metaData) .routingTable(routingTable) .nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build(); - return new RoutingAllocation(allocationDeciders, new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + return new RoutingAllocation(allocationDeciders, new RoutingNodes(state, false), state, null, System.nanoTime()); } /** @@ -425,7 +425,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { .metaData(metaData) .routingTable(routingTable) .nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build(); - return new RoutingAllocation(allocationDeciders, new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + return new RoutingAllocation(allocationDeciders, new RoutingNodes(state, false), state, null, System.nanoTime()); } /** @@ -444,7 +444,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { .routingTable(routingTable) .nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build(); - RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime()); boolean changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(false)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); @@ -452,7 +452,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas testAllocator.addData(node1, 1, null, randomBoolean()); - allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime()); changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(false)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); @@ -460,7 +460,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas testAllocator.addData(node2, 1, null, randomBoolean()); - allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime()); changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); @@ -485,7 +485,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { .routingTable(routingTable) .nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build(); - RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime()); boolean changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(false)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); @@ -493,7 +493,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas testAllocator.addData(node1, 1, null, randomBoolean()); - allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime()); changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(false)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); @@ -501,7 +501,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas testAllocator.addData(node2, 2, null, randomBoolean()); - allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime()); changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); @@ -525,7 +525,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { .metaData(metaData) .routingTable(routingTableBuilder.build()) .nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build(); - return new RoutingAllocation(deciders, new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); + return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, null, System.nanoTime()); } class TestAllocator extends PrimaryShardAllocator { diff --git a/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java b/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java index 96f81e2ab73..672c9de3d3e 100644 --- a/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java @@ -302,7 +302,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase { .metaData(metaData) .routingTable(routingTable) .nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build(); - return new RoutingAllocation(deciders, new RoutingNodes(state, false), state.nodes(), ClusterInfo.EMPTY, System.nanoTime()); + return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, ClusterInfo.EMPTY, System.nanoTime()); } private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders) { @@ -324,7 +324,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase { .metaData(metaData) .routingTable(routingTable) .nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build(); - return new RoutingAllocation(deciders, new RoutingNodes(state, false), state.nodes(), ClusterInfo.EMPTY, System.nanoTime()); + return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, ClusterInfo.EMPTY, System.nanoTime()); } class TestAllocator extends ReplicaShardAllocator { diff --git a/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java b/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java index 0b1723bd5aa..dc533737886 100644 --- a/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java +++ b/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java @@ -103,7 +103,7 @@ public class RareClusterStateIT extends ESIntegTestCase { .nodes(DiscoveryNodes.EMPTY_NODES) .build(), false ); - RoutingAllocation routingAllocation = new RoutingAllocation(allocationDeciders, routingNodes, current.nodes(), ClusterInfo.EMPTY, System.nanoTime()); + RoutingAllocation routingAllocation = new RoutingAllocation(allocationDeciders, routingNodes, current, ClusterInfo.EMPTY, System.nanoTime()); allocator.allocateUnassigned(routingAllocation); } diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SortProcessorTests.java b/core/src/test/java/org/elasticsearch/ingest/processor/SortProcessorTests.java new file mode 100644 index 00000000000..18e35d00a79 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/ingest/processor/SortProcessorTests.java @@ -0,0 +1,282 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.ingest.processor; + +import org.elasticsearch.ingest.core.IngestDocument; +import org.elasticsearch.ingest.RandomDocumentPicks; +import org.elasticsearch.ingest.core.Processor; +import org.elasticsearch.ingest.processor.SortProcessor.SortOrder; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Random; +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class SortProcessorTests extends ESTestCase { + + public void testSortStrings() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(numItems); + List expectedResult = new ArrayList<>(numItems); + for (int j = 0; j < numItems; j++) { + String value = randomAsciiOfLengthBetween(1, 10); + fieldValue.add(value); + expectedResult.add(value); + } + Collections.sort(expectedResult); + + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + if (order.equals(SortOrder.DESCENDING)) { + Collections.reverse(expectedResult); + } + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + processor.execute(ingestDocument); + assertEquals(ingestDocument.getFieldValue(fieldName, List.class), expectedResult); + } + + public void testSortIntegersNonRandom() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + + Integer[] expectedResult = new Integer[]{1,2,3,4,5,10,20,21,22,50,100}; + List fieldValue = new ArrayList<>(expectedResult.length); + fieldValue.addAll(Arrays.asList(expectedResult).subList(0, expectedResult.length)); + Collections.shuffle(fieldValue, random()); + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, SortOrder.ASCENDING); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(fieldName, List.class).toArray(), equalTo(expectedResult)); + } + + public void testSortIntegers() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(numItems); + List expectedResult = new ArrayList<>(numItems); + for (int j = 0; j < numItems; j++) { + Integer value = randomIntBetween(1, 100); + fieldValue.add(value); + expectedResult.add(value); + } + Collections.sort(expectedResult); + + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + if (order.equals(SortOrder.DESCENDING)) { + Collections.reverse(expectedResult); + } + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + processor.execute(ingestDocument); + assertEquals(ingestDocument.getFieldValue(fieldName, List.class), expectedResult); + } + + public void testSortShorts() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(numItems); + List expectedResult = new ArrayList<>(numItems); + for (int j = 0; j < numItems; j++) { + Short value = randomShort(); + fieldValue.add(value); + expectedResult.add(value); + } + Collections.sort(expectedResult); + + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + if (order.equals(SortOrder.DESCENDING)) { + Collections.reverse(expectedResult); + } + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + processor.execute(ingestDocument); + assertEquals(ingestDocument.getFieldValue(fieldName, List.class), expectedResult); + } + + public void testSortDoubles() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(numItems); + List expectedResult = new ArrayList<>(numItems); + for (int j = 0; j < numItems; j++) { + Double value = randomDoubleBetween(0.0, 100.0, true); + fieldValue.add(value); + expectedResult.add(value); + } + Collections.sort(expectedResult); + + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + if (order.equals(SortOrder.DESCENDING)) { + Collections.reverse(expectedResult); + } + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + processor.execute(ingestDocument); + assertEquals(ingestDocument.getFieldValue(fieldName, List.class), expectedResult); + } + + public void testSortFloats() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(numItems); + List expectedResult = new ArrayList<>(numItems); + for (int j = 0; j < numItems; j++) { + Float value = randomFloat(); + fieldValue.add(value); + expectedResult.add(value); + } + Collections.sort(expectedResult); + + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + if (order.equals(SortOrder.DESCENDING)) { + Collections.reverse(expectedResult); + } + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + processor.execute(ingestDocument); + assertEquals(ingestDocument.getFieldValue(fieldName, List.class), expectedResult); + } + + public void testSortBytes() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(numItems); + List expectedResult = new ArrayList<>(numItems); + for (int j = 0; j < numItems; j++) { + Byte value = randomByte(); + fieldValue.add(value); + expectedResult.add(value); + } + Collections.sort(expectedResult); + + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + if (order.equals(SortOrder.DESCENDING)) { + Collections.reverse(expectedResult); + } + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + processor.execute(ingestDocument); + assertEquals(ingestDocument.getFieldValue(fieldName, List.class), expectedResult); + } + + public void testSortBooleans() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(numItems); + List expectedResult = new ArrayList<>(numItems); + for (int j = 0; j < numItems; j++) { + Boolean value = randomBoolean(); + fieldValue.add(value); + expectedResult.add(value); + } + Collections.sort(expectedResult); + + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + if (order.equals(SortOrder.DESCENDING)) { + Collections.reverse(expectedResult); + } + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + processor.execute(ingestDocument); + assertEquals(ingestDocument.getFieldValue(fieldName, List.class), expectedResult); + } + + public void testSortMixedStrings() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + int numItems = randomIntBetween(1, 10); + List fieldValue = new ArrayList<>(numItems); + List expectedResult = new ArrayList<>(numItems); + String value; + for (int j = 0; j < numItems; j++) { + if (randomBoolean()) { + value = String.valueOf(randomIntBetween(0, 100)); + } else { + value = randomAsciiOfLengthBetween(1, 10); + } + fieldValue.add(value); + expectedResult.add(value); + } + Collections.sort(expectedResult); + + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + if (order.equals(SortOrder.DESCENDING)) { + Collections.reverse(expectedResult); + } + + String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, fieldValue); + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + processor.execute(ingestDocument); + assertEquals(ingestDocument.getFieldValue(fieldName, List.class), expectedResult); + } + + public void testSortNonListField() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + ingestDocument.setFieldValue(fieldName, randomAsciiOfLengthBetween(1, 10)); + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + try { + processor.execute(ingestDocument); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.String] cannot be cast to [java.util.List]")); + } + } + + public void testSortNonExistingField() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>()); + String fieldName = RandomDocumentPicks.randomFieldName(random()); + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + Processor processor = new SortProcessor(randomAsciiOfLength(10), fieldName, order); + try { + processor.execute(ingestDocument); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("not present as part of path [" + fieldName + "]")); + } + } + + public void testSortNullValue() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), Collections.singletonMap("field", null)); + SortOrder order = randomBoolean() ? SortOrder.ASCENDING : SortOrder.DESCENDING; + Processor processor = new SortProcessor(randomAsciiOfLength(10), "field", order); + try { + processor.execute(ingestDocument); + } catch(IllegalArgumentException e) { + assertThat(e.getMessage(), equalTo("field [field] is null, cannot sort.")); + } + } + + + +} diff --git a/core/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceTests.java b/core/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceTests.java new file mode 100644 index 00000000000..2c17fca7c8c --- /dev/null +++ b/core/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceTests.java @@ -0,0 +1,136 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.monitor.jvm; + +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.test.ESTestCase; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +public class JvmGcMonitorServiceTests extends ESTestCase { + + public void testSlowGcLogging() { + final ESLogger logger = mock(ESLogger.class); + when(logger.isWarnEnabled()).thenReturn(true); + when(logger.isInfoEnabled()).thenReturn(true); + when(logger.isDebugEnabled()).thenReturn(true); + final JvmGcMonitorService.JvmMonitor.Threshold threshold = randomFrom(JvmGcMonitorService.JvmMonitor.Threshold.values()); + final String name = randomAsciiOfLength(16); + final long seq = randomIntBetween(1, 1 << 30); + final int elapsedValue = randomIntBetween(1, 1 << 10); + final long totalCollectionCount = randomIntBetween(1, 16); + final long currentCollectionCount = randomIntBetween(1, 16); + final TimeValue totalCollectionTime = TimeValue.timeValueMillis(randomIntBetween(1, elapsedValue)); + final TimeValue currentCollectionTime = TimeValue.timeValueMillis(randomIntBetween(1, elapsedValue)); + + final ByteSizeValue lastHeapUsed = new ByteSizeValue(randomIntBetween(1, 1 << 10)); + JvmStats lastJvmStats = mock(JvmStats.class); + JvmStats.Mem lastMem = mock(JvmStats.Mem.class); + when(lastMem.getHeapUsed()).thenReturn(lastHeapUsed); + when(lastJvmStats.getMem()).thenReturn(lastMem); + when(lastJvmStats.toString()).thenReturn("last"); + + final ByteSizeValue currentHeapUsed = new ByteSizeValue(randomIntBetween(1, 1 << 10)); + JvmStats currentJvmStats = mock(JvmStats.class); + JvmStats.Mem currentMem = mock(JvmStats.Mem.class); + when(currentMem.getHeapUsed()).thenReturn(currentHeapUsed); + when(currentJvmStats.getMem()).thenReturn(currentMem); + when(currentJvmStats.toString()).thenReturn("current"); + + JvmStats.GarbageCollector gc = mock(JvmStats.GarbageCollector.class); + when(gc.getName()).thenReturn(name); + when(gc.getCollectionCount()).thenReturn(totalCollectionCount); + when(gc.getCollectionTime()).thenReturn(totalCollectionTime); + + final ByteSizeValue maxHeapUsed = new ByteSizeValue(Math.max(lastHeapUsed.bytes(), currentHeapUsed.bytes()) + 1 << 10); + + JvmGcMonitorService.JvmMonitor.SlowGcEvent slowGcEvent = new JvmGcMonitorService.JvmMonitor.SlowGcEvent( + gc, + currentCollectionCount, + currentCollectionTime, + elapsedValue, + lastJvmStats, + currentJvmStats, + maxHeapUsed); + + JvmGcMonitorService.logSlowGc(logger, threshold, seq, slowGcEvent, (l, c) -> l.toString() + ", " + c.toString()); + + switch (threshold) { + case WARN: + verify(logger).isWarnEnabled(); + verify(logger).warn( + "[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}", + name, + seq, + totalCollectionCount, + currentCollectionTime, + currentCollectionCount, + TimeValue.timeValueMillis(elapsedValue), + currentCollectionTime, + totalCollectionTime, + lastHeapUsed, + currentHeapUsed, + maxHeapUsed, + "last, current"); + break; + case INFO: + verify(logger).isInfoEnabled(); + verify(logger).info( + "[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}", + name, + seq, + totalCollectionCount, + currentCollectionTime, + currentCollectionCount, + TimeValue.timeValueMillis(elapsedValue), + currentCollectionTime, + totalCollectionTime, + lastHeapUsed, + currentHeapUsed, + maxHeapUsed, + "last, current"); + break; + case DEBUG: + verify(logger).isDebugEnabled(); + verify(logger).debug( + "[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}", + name, + seq, + totalCollectionCount, + currentCollectionTime, + currentCollectionCount, + TimeValue.timeValueMillis(elapsedValue), + currentCollectionTime, + totalCollectionTime, + lastHeapUsed, + currentHeapUsed, + maxHeapUsed, + "last, current"); + break; + } + verifyNoMoreInteractions(logger); + } + +} diff --git a/core/src/test/java/org/elasticsearch/monitor/jvm/JvmMonitorTests.java b/core/src/test/java/org/elasticsearch/monitor/jvm/JvmMonitorTests.java new file mode 100644 index 00000000000..8d3ddeec84e --- /dev/null +++ b/core/src/test/java/org/elasticsearch/monitor/jvm/JvmMonitorTests.java @@ -0,0 +1,248 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.monitor.jvm; + +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static org.hamcrest.CoreMatchers.anyOf; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.Matchers.hasToString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class JvmMonitorTests extends ESTestCase { + + public void testMonitorFailure() { + AtomicBoolean shouldFail = new AtomicBoolean(); + AtomicBoolean invoked = new AtomicBoolean(); + JvmGcMonitorService.JvmMonitor monitor = new JvmGcMonitorService.JvmMonitor(Collections.emptyMap()) { + @Override + void onMonitorFailure(Throwable t) { + invoked.set(true); + assertThat(t, instanceOf(RuntimeException.class)); + assertThat(t, hasToString(containsString("simulated"))); + } + + @Override + synchronized void monitorLongGc() { + if (shouldFail.get()) { + throw new RuntimeException("simulated"); + } + } + + @Override + void onSlowGc(final Threshold threshold, final long seq, final SlowGcEvent slowGcEvent) { + } + }; + + monitor.run(); + assertFalse(invoked.get()); + + shouldFail.set(true); + monitor.run(); + assertTrue(invoked.get()); + } + + public void testSlowGc() { + + final int initialYoungCollectionCount = randomIntBetween(1, 4); + final int initialYoungCollectionTime = randomIntBetween(initialYoungCollectionCount * 100, initialYoungCollectionCount * 200); + + final int initialOldCollectionCount = randomIntBetween(1, 4); + final int initialOldCollectionTime = randomIntBetween(initialYoungCollectionCount * 1000, initialYoungCollectionCount * 2000); + + final JvmStats.GarbageCollector initialYoungCollector = mock(JvmStats.GarbageCollector.class); + when(initialYoungCollector.getName()).thenReturn("young"); + when(initialYoungCollector.getCollectionCount()).thenReturn((long) initialYoungCollectionCount); + when(initialYoungCollector.getCollectionTime()).thenReturn(TimeValue.timeValueMillis(initialYoungCollectionTime)); + + final JvmStats.GarbageCollector initialOldCollector = mock(JvmStats.GarbageCollector.class); + when(initialOldCollector.getName()).thenReturn("old"); + when(initialOldCollector.getCollectionCount()).thenReturn((long) initialOldCollectionCount); + when(initialOldCollector.getCollectionTime()).thenReturn(TimeValue.timeValueMillis(initialOldCollectionTime)); + JvmStats initialJvmStats = jvmStats(initialYoungCollector, initialOldCollector); + + final Map gcThresholds = new HashMap<>(); + + // fake debug threshold, info will be double this and warn will + // be triple + final int youngDebugThreshold = randomIntBetween(1, 10) * 100; + gcThresholds.put( + "young", + new JvmGcMonitorService.GcThreshold("young", youngDebugThreshold * 3, youngDebugThreshold * 2, youngDebugThreshold)); + + final boolean youngGcThreshold = randomBoolean(); + final JvmGcMonitorService.JvmMonitor.Threshold youngThresholdLevel = randomFrom(JvmGcMonitorService.JvmMonitor.Threshold.values()); + final int youngMultiplier = 1 + youngThresholdLevel.ordinal(); + final int youngCollections = randomIntBetween(1, 4); + + final JvmStats.GarbageCollector youngCollector; + youngCollector = mock(JvmStats.GarbageCollector.class); + when(youngCollector.getName()).thenReturn("young"); + when(youngCollector.getCollectionCount()).thenReturn((long) (initialYoungCollectionCount + youngCollections)); + + final int youngIncrement; + if (youngGcThreshold) { + // we are faking that youngCollections collections occurred + // this number is chosen so that we squeak over the + // random threshold when computing the average collection + // time: note that average collection time will just be + // youngMultiplier * youngDebugThreshold + 1 which ensures + // that we are over the right threshold but below the next + // threshold + youngIncrement = youngCollections * youngMultiplier * youngDebugThreshold + youngCollections; + } else { + // fake that we did not exceed the threshold + youngIncrement = randomIntBetween(1, youngDebugThreshold); + } + when(youngCollector.getCollectionTime()).thenReturn(TimeValue.timeValueMillis(initialYoungCollectionTime + youngIncrement)); + + // fake debug threshold, info will be double this and warn will + // be triple + final int oldDebugThreshold = randomIntBetween(1, 10) * 100; + gcThresholds.put( + "old", + new JvmGcMonitorService.GcThreshold("old", oldDebugThreshold * 3, oldDebugThreshold * 2, oldDebugThreshold)); + + final boolean oldGcThreshold = randomBoolean(); + final JvmGcMonitorService.JvmMonitor.Threshold oldThresholdLevel = randomFrom(JvmGcMonitorService.JvmMonitor.Threshold.values()); + final int oldMultiplier = 1 + oldThresholdLevel.ordinal(); + final int oldCollections = randomIntBetween(1, 4); + + final JvmStats.GarbageCollector oldCollector = mock(JvmStats.GarbageCollector.class); + when(oldCollector.getName()).thenReturn("old"); + when(oldCollector.getCollectionCount()).thenReturn((long) (initialOldCollectionCount + oldCollections)); + final int oldIncrement; + if (oldGcThreshold) { + // we are faking that oldCollections collections occurred + // this number is chosen so that we squeak over the + // random threshold when computing the average collection + // time: note that average collection time will just be + // oldMultiplier * oldDebugThreshold + 1 which ensures + // that we are over the right threshold but below the next + // threshold + oldIncrement = oldCollections * oldMultiplier * oldDebugThreshold + oldCollections; + } else { + // fake that we did not exceed the threshold + oldIncrement = randomIntBetween(1, oldDebugThreshold); + } + when(oldCollector.getCollectionTime()).thenReturn(TimeValue.timeValueMillis(initialOldCollectionTime + oldIncrement)); + + final long start = randomIntBetween(1, 1 << 30); + final long expectedElapsed = randomIntBetween(1, 1000); + final AtomicLong now = new AtomicLong(start); + + final AtomicReference jvmStats = new AtomicReference<>(); + jvmStats.set(initialJvmStats); + + final AtomicInteger count = new AtomicInteger(); + + JvmGcMonitorService.JvmMonitor monitor = new JvmGcMonitorService.JvmMonitor(gcThresholds) { + @Override + void onMonitorFailure(Throwable t) { + } + + @Override + void onSlowGc(final Threshold threshold, final long seq, final SlowGcEvent slowGcEvent) { + count.incrementAndGet(); + assertThat(seq, equalTo(1L)); + assertThat(slowGcEvent.elapsed, equalTo(expectedElapsed)); + assertThat(slowGcEvent.currentGc.getName(), anyOf(equalTo("young"), equalTo("old"))); + if ("young".equals(slowGcEvent.currentGc.getName())) { + assertCollection( + threshold, + youngThresholdLevel, + slowGcEvent, + initialYoungCollectionCount, + youngCollections, + initialYoungCollectionTime, + youngIncrement); + } else if ("old".equals(slowGcEvent.currentGc.getName())) { + assertCollection( + threshold, + oldThresholdLevel, + slowGcEvent, + initialOldCollectionCount, + oldCollections, + initialOldCollectionTime, + oldIncrement); + } + } + + @Override + long now() { + return now.get(); + } + + @Override + JvmStats jvmStats() { + return jvmStats.get(); + } + }; + + final JvmStats monitorJvmStats = jvmStats(youngCollector, oldCollector); + + now.set(start + TimeUnit.NANOSECONDS.convert(expectedElapsed, TimeUnit.MILLISECONDS)); + jvmStats.set(monitorJvmStats); + monitor.monitorLongGc(); + + assertThat(count.get(), equalTo((youngGcThreshold ? 1 : 0) + (oldGcThreshold ? 1 : 0))); + } + + private void assertCollection( + final JvmGcMonitorService.JvmMonitor.Threshold actualThreshold, + final JvmGcMonitorService.JvmMonitor.Threshold expectedThreshold, + final JvmGcMonitorService.JvmMonitor.SlowGcEvent slowGcEvent, + final int initialCollectionCount, + final int collections, + final int initialCollectionTime, + final int increment) { + assertThat(actualThreshold, equalTo(expectedThreshold)); + assertThat(slowGcEvent.currentGc.getCollectionCount(), equalTo((long) (initialCollectionCount + collections))); + assertThat(slowGcEvent.collectionCount, equalTo((long) collections)); + assertThat(slowGcEvent.collectionTime, equalTo(TimeValue.timeValueMillis(increment))); + assertThat(slowGcEvent.currentGc.getCollectionTime(), equalTo(TimeValue.timeValueMillis(initialCollectionTime + increment))); + } + + private JvmStats jvmStats(JvmStats.GarbageCollector youngCollector, JvmStats.GarbageCollector oldCollector) { + final JvmStats jvmStats = mock(JvmStats.class); + final JvmStats.GarbageCollectors initialGcs = mock(JvmStats.GarbageCollectors.class); + final JvmStats.GarbageCollector[] initialCollectors = new JvmStats.GarbageCollector[2]; + initialCollectors[0] = youngCollector; + initialCollectors[1] = oldCollector; + when(initialGcs.getCollectors()).thenReturn(initialCollectors); + when(jvmStats.getGc()).thenReturn(initialGcs); + when(jvmStats.getMem()).thenReturn(JvmStats.jvmStats().getMem()); + return jvmStats; + } + +} diff --git a/core/src/test/java/org/elasticsearch/search/profile/ProfileTests.java b/core/src/test/java/org/elasticsearch/search/profile/query/ProfileTests.java similarity index 97% rename from core/src/test/java/org/elasticsearch/search/profile/ProfileTests.java rename to core/src/test/java/org/elasticsearch/search/profile/query/ProfileTests.java index ebae8e64f32..7488dbceee7 100644 --- a/core/src/test/java/org/elasticsearch/search/profile/ProfileTests.java +++ b/core/src/test/java/org/elasticsearch/search/profile/query/ProfileTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field.Store; @@ -37,6 +37,9 @@ import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.TestUtil; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.search.internal.ContextIndexSearcher; +import org.elasticsearch.search.profile.ProfileResult; +import org.elasticsearch.search.profile.query.QueryProfiler; +import org.elasticsearch.search.profile.query.QueryTimingType; import org.elasticsearch.test.ESTestCase; import org.junit.AfterClass; import org.junit.BeforeClass; diff --git a/core/src/test/java/org/elasticsearch/search/profile/QueryProfilerIT.java b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java similarity index 97% rename from core/src/test/java/org/elasticsearch/search/profile/QueryProfilerIT.java rename to core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java index d9878774576..371aaadd3a7 100644 --- a/core/src/test/java/org/elasticsearch/search/profile/QueryProfilerIT.java +++ b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.util.English; import org.elasticsearch.action.index.IndexRequestBuilder; @@ -29,6 +29,9 @@ import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.profile.ProfileResult; +import org.elasticsearch.search.profile.ProfileShardResult; +import org.elasticsearch.search.profile.query.CollectorResult; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; @@ -36,7 +39,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import static org.elasticsearch.search.profile.RandomQueryGenerator.randomQueryBuilder; +import static org.elasticsearch.search.profile.query.RandomQueryGenerator.randomQueryBuilder; import static org.elasticsearch.test.hamcrest.DoubleMatcher.nearlyEqual; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -160,7 +163,8 @@ public class QueryProfilerIT extends ESIntegTestCase { nearlyEqual(vanillaMaxScore, profileMaxScore, 0.001)); } - assertThat("Profile totalHits of [" + profileResponse.getHits().totalHits() + "] is not close to Vanilla totalHits [" + vanillaResponse.getHits().totalHits() + "]", + assertThat("Profile totalHits of [" + profileResponse.getHits().totalHits() + "] is not close to Vanilla totalHits [" + + vanillaResponse.getHits().totalHits() + "]", vanillaResponse.getHits().getTotalHits(), equalTo(profileResponse.getHits().getTotalHits())); SearchHit[] vanillaHits = vanillaResponse.getHits().getHits(); @@ -237,7 +241,8 @@ public class QueryProfilerIT extends ESIntegTestCase { indexRandom(true, docs); - QueryBuilder q = QueryBuilders.boolQuery().must(QueryBuilders.matchQuery("field1", "one")).must(QueryBuilders.matchQuery("field1", "two")); + QueryBuilder q = QueryBuilders.boolQuery().must(QueryBuilders.matchQuery("field1", "one")) + .must(QueryBuilders.matchQuery("field1", "two")); SearchResponse resp = client().prepareSearch() .setQuery(q) @@ -355,7 +360,8 @@ public class QueryProfilerIT extends ESIntegTestCase { refresh(); - QueryBuilder q = QueryBuilders.boolQuery().must(QueryBuilders.boolQuery().must(QueryBuilders.boolQuery().must(QueryBuilders.matchQuery("field1", "one")))); + QueryBuilder q = QueryBuilders.boolQuery() + .must(QueryBuilders.boolQuery().must(QueryBuilders.boolQuery().must(QueryBuilders.matchQuery("field1", "one")))); logger.info("Query: {}", q); diff --git a/core/src/test/java/org/elasticsearch/search/profile/RandomQueryGenerator.java b/core/src/test/java/org/elasticsearch/search/profile/query/RandomQueryGenerator.java similarity index 99% rename from core/src/test/java/org/elasticsearch/search/profile/RandomQueryGenerator.java rename to core/src/test/java/org/elasticsearch/search/profile/query/RandomQueryGenerator.java index fff0b99c4d1..7fb896e1b2f 100644 --- a/core/src/test/java/org/elasticsearch/search/profile/RandomQueryGenerator.java +++ b/core/src/test/java/org/elasticsearch/search/profile/query/RandomQueryGenerator.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.search.profile; +package org.elasticsearch.search.profile.query; import org.apache.lucene.util.English; import org.elasticsearch.common.unit.Fuzziness; diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index 790476990d9..bde6ad8b611 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -33,7 +33,10 @@ That will return something like this: "batches": 1, "version_conflicts": 0, "noops": 0, - "retries": 0, + "retries": { + "bulk": 0, + "search": 0 + }, "throttled_millis": 0, "requests_per_second": "unlimited", "throttled_until_millis": 0, @@ -386,7 +389,10 @@ The JSON response looks like this: "created": 123, "batches": 1, "version_conflicts": 2, - "retries": 0, + "retries": { + "bulk": 0, + "search": 0 + } "throttled_millis": 0, "failures" : [ ] } @@ -414,7 +420,8 @@ The number of version conflicts that reindex hit. `retries`:: -The number of retries that the reindex did in response to a full queue. +The number of retries attempted by reindex. `bulk` is the number of bulk +actions retried and `search` is the number of search actions retried. `throttled_millis`:: @@ -468,7 +475,10 @@ The responses looks like: "batches" : 4, "version_conflicts" : 0, "noops" : 0, - "retries": 0, + "retries": { + "bulk": 0, + "search": 0 + }, "throttled_millis": 0 }, "description" : "" diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index c3eb0908398..e5a62ddf33a 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -26,7 +26,10 @@ That will return something like this: "batches": 1, "version_conflicts": 0, "noops": 0, - "retries": 0, + "retries": { + "bulk": 0, + "search": 0 + }, "throttled_millis": 0, "requests_per_second": "unlimited", "throttled_until_millis": 0, @@ -220,7 +223,10 @@ The JSON response looks like this: "updated": 0, "batches": 1, "version_conflicts": 2, - "retries": 0, + "retries": { + "bulk": 0, + "search": 0 + } "throttled_millis": 0, "failures" : [ ] } @@ -244,7 +250,8 @@ The number of version conflicts that the update by query hit. `retries`:: -The number of retries that the update by query did in response to a full queue. +The number of retries attempted by update-by-query. `bulk` is the number of bulk +actions retried and `search` is the number of search actions retried. `throttled_millis`:: @@ -299,7 +306,10 @@ The responses looks like: "batches" : 4, "version_conflicts" : 0, "noops" : 0, - "retries": 0, + "retries": { + "bulk": 0, + "search": 0 + } "throttled_millis": 0 }, "description" : "" diff --git a/docs/reference/ingest/ingest-node.asciidoc b/docs/reference/ingest/ingest-node.asciidoc index ec6253f2d93..3dcf719f382 100644 --- a/docs/reference/ingest/ingest-node.asciidoc +++ b/docs/reference/ingest/ingest-node.asciidoc @@ -1282,6 +1282,31 @@ Splits a field into an array using a separator character. Only works on string f -------------------------------------------------- <1> Treat all consecutive whitespace characters as a single separator +[[sort-processor]] +=== Sort Processor +Sorts the elements of an array ascending or descending. Homogeneous arrays of numbers will be sorted +numerically, while arrays of strings or heterogeneous arrays of strings + numbers will be sorted lexicographically. +Throws an error when the field is not an array. + +[[sort-options]] +.Sort Options +[options="header"] +|====== +| Name | Required | Default | Description +| `field` | yes | - | The field to be sorted +| `order` | no | `"asc"` | The sort order to use. Accepts `"asc"` or `"desc"`. +|====== + +[source,js] +-------------------------------------------------- +{ + "sort": { + "field": "field_to_sort", + "order": "desc" + } +} +-------------------------------------------------- + [[trim-processor]] === Trim Processor Trims whitespace from field. diff --git a/docs/reference/migration/migrate_5_0.asciidoc b/docs/reference/migration/migrate_5_0.asciidoc index e2dae648485..71e37e49581 100644 --- a/docs/reference/migration/migrate_5_0.asciidoc +++ b/docs/reference/migration/migrate_5_0.asciidoc @@ -34,6 +34,7 @@ way to do this is to upgrade to Elasticsearch 2.3 or later and to use the * <> * <> * <> +* <> * <> * <> * <> diff --git a/docs/reference/migration/migrate_5_0/docs.asciidoc b/docs/reference/migration/migrate_5_0/docs.asciidoc new file mode 100644 index 00000000000..85e4e901e5c --- /dev/null +++ b/docs/reference/migration/migrate_5_0/docs.asciidoc @@ -0,0 +1,33 @@ +[[breaking_50_document_api_changes]] +=== Document API changes + +==== Reindex and Update By Query +Before 5.0.0 `_reindex` and `_update_by_query` only retried bulk failures so +they used the following response format: + +[source,js] +---------------------- +{ + ... + "retries": 10 + ... +} +---------------------- + +Where `retries` counts the number of bulk retries. Now they retry on search +failures as well and use this response format: + +[source,js] +---------------------- +{ + ... + "retries": { + "bulk": 10, + "search": 1 + } + ... +} +---------------------- + +Where `bulk` counts the number of bulk retries and `search` counts the number +of search retries. diff --git a/docs/reference/modules/discovery/zen.asciidoc b/docs/reference/modules/discovery/zen.asciidoc index 12de0e3b736..60823393d11 100644 --- a/docs/reference/modules/discovery/zen.asciidoc +++ b/docs/reference/modules/discovery/zen.asciidoc @@ -52,7 +52,7 @@ with a timeout defaulting at 20 times the ping timeout. When the master node stops or has encountered a problem, the cluster nodes start pinging again and will elect a new master. This pinging round also -serves as a protection against (partial) network failures where node may unjustly +serves as a protection against (partial) network failures where a node may unjustly think that the master has failed. In this case the node will simply hear from other nodes about the currently active master. diff --git a/docs/reference/modules/scripting/painless.asciidoc b/docs/reference/modules/scripting/painless.asciidoc index 156705bfe78..d79b59cf0d6 100644 --- a/docs/reference/modules/scripting/painless.asciidoc +++ b/docs/reference/modules/scripting/painless.asciidoc @@ -189,7 +189,7 @@ POST hockey/player/1/_update { "script": { "lang": "painless", - "inline": "ctx._source.last = params.last ctx._source.nick = params.nick", + "inline": "ctx._source.last = params.last; ctx._source.nick = params.nick", "params": { "last": "gaudreau", "nick": "hockey" diff --git a/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml b/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml index f61027c3de3..c41793fcdb8 100644 --- a/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml +++ b/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml @@ -22,6 +22,7 @@ - match: { nodes.$master.ingest.processors.10.type: remove } - match: { nodes.$master.ingest.processors.11.type: rename } - match: { nodes.$master.ingest.processors.12.type: set } - - match: { nodes.$master.ingest.processors.13.type: split } - - match: { nodes.$master.ingest.processors.14.type: trim } - - match: { nodes.$master.ingest.processors.15.type: uppercase } + - match: { nodes.$master.ingest.processors.13.type: sort } + - match: { nodes.$master.ingest.processors.14.type: split } + - match: { nodes.$master.ingest.processors.15.type: trim } + - match: { nodes.$master.ingest.processors.16.type: uppercase } diff --git a/modules/lang-painless/src/main/antlr/PainlessParser.g4 b/modules/lang-painless/src/main/antlr/PainlessParser.g4 index e8aa3592c59..4d9a81e25a0 100644 --- a/modules/lang-painless/src/main/antlr/PainlessParser.g4 +++ b/modules/lang-painless/src/main/antlr/PainlessParser.g4 @@ -28,15 +28,15 @@ source statement : IF LP expression RP block ( ELSE block )? # if | WHILE LP expression RP ( block | empty ) # while - | DO block WHILE LP expression RP SEMICOLON? # do + | DO block WHILE LP expression RP ( SEMICOLON | EOF ) # do | FOR LP initializer? SEMICOLON expression? SEMICOLON afterthought? RP ( block | empty ) # for - | declaration SEMICOLON? # decl - | CONTINUE SEMICOLON? # continue - | BREAK SEMICOLON? # break - | RETURN expression SEMICOLON? # return + | declaration ( SEMICOLON | EOF ) # decl + | CONTINUE ( SEMICOLON | EOF ) # continue + | BREAK ( SEMICOLON | EOF ) # break + | RETURN expression ( SEMICOLON | EOF ) # return | TRY block trap+ # try - | THROW expression SEMICOLON? # throw - | expression SEMICOLON? # expr + | THROW expression ( SEMICOLON | EOF ) # throw + | expression ( SEMICOLON | EOF ) # expr ; block diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java index f82f04d801a..d9b2e02a237 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java @@ -133,37 +133,41 @@ public final class Def { *

* @param receiverClass Class of the object to invoke the method on. * @param name Name of the method. + * @param type Callsite signature. Need not match exactly, except the number of parameters. * @param definition Whitelist to check. * @return pointer to matching method to invoke. never returns null. * @throws IllegalArgumentException if no matching whitelisted method was found. */ - static MethodHandle lookupMethod(Class receiverClass, String name, Definition definition) { - // check whitelist for matching method - for (Class clazz = receiverClass; clazz != null; clazz = clazz.getSuperclass()) { - RuntimeClass struct = definition.runtimeMap.get(clazz); - - if (struct != null) { - Method method = struct.methods.get(name); - if (method != null) { - return method.handle; - } - } - - for (final Class iface : clazz.getInterfaces()) { - struct = definition.runtimeMap.get(iface); - - if (struct != null) { - Method method = struct.methods.get(name); - if (method != null) { - return method.handle; - } - } - } - } - - // no matching methods in whitelist found - throw new IllegalArgumentException("Unable to find dynamic method [" + name + "] " + - "for class [" + receiverClass.getCanonicalName() + "]."); + static MethodHandle lookupMethod(Class receiverClass, String name, MethodType type, Definition definition) { + // we don't consider receiver an argument/counting towards arity + type = type.dropParameterTypes(0, 1); + Definition.MethodKey key = new Definition.MethodKey(name, type.parameterCount()); + // check whitelist for matching method + for (Class clazz = receiverClass; clazz != null; clazz = clazz.getSuperclass()) { + RuntimeClass struct = definition.runtimeMap.get(clazz); + + if (struct != null) { + Method method = struct.methods.get(key); + if (method != null) { + return method.handle; + } + } + + for (final Class iface : clazz.getInterfaces()) { + struct = definition.runtimeMap.get(iface); + + if (struct != null) { + Method method = struct.methods.get(key); + if (method != null) { + return method.handle; + } + } + } + } + + // no matching methods in whitelist found + throw new IllegalArgumentException("Unable to find dynamic method [" + name + "] with signature [" + type + "] " + + "for class [" + receiverClass.getCanonicalName() + "]."); } /** diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java index d9e96302eec..380f5455ab3 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java @@ -91,10 +91,10 @@ public final class DefBootstrap { /** * Does a slow lookup against the whitelist. */ - private static MethodHandle lookup(int flavor, Class clazz, String name) { + private static MethodHandle lookup(int flavor, Class clazz, String name, MethodType type) { switch(flavor) { case METHOD_CALL: - return Def.lookupMethod(clazz, name, Definition.INSTANCE); + return Def.lookupMethod(clazz, name, type, Definition.INSTANCE); case LOAD: return Def.lookupGetter(clazz, name, Definition.INSTANCE); case STORE: @@ -115,7 +115,7 @@ public final class DefBootstrap { final MethodType type = type(); final Object receiver = args[0]; final Class receiverClass = receiver.getClass(); - final MethodHandle target = lookup(flavor, receiverClass, name).asType(type); + final MethodHandle target = lookup(flavor, receiverClass, name, type).asType(type); if (depth >= MAX_DEPTH) { // revert to a vtable call diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Definition.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Definition.java index 6a55b2f5042..40dc13a3368 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Definition.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Definition.java @@ -33,6 +33,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; /** @@ -194,17 +195,72 @@ public final class Definition { this.setter = setter; } } + + // TODO: instead of hashing on this, we could have a 'next' pointer in Method itself, but it would make code more complex + // please do *NOT* under any circumstances change this to be the crappy Tuple from elasticsearch! + /** + * Key for looking up a method. + *

+ * Methods are keyed on both name and arity, and can be overloaded once per arity. + * This allows signatures such as {@code String.indexOf(String) vs String.indexOf(String, int)}. + *

+ * It is less flexible than full signature overloading where types can differ too, but + * better than just the name, and overloading types adds complexity to users, too. + */ + public static final class MethodKey { + public final String name; + public final int arity; + + /** + * Create a new lookup key + * @param name name of the method + * @param arity number of parameters + */ + public MethodKey(String name, int arity) { + this.name = Objects.requireNonNull(name); + this.arity = arity; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + arity; + result = prime * result + name.hashCode(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null) return false; + if (getClass() != obj.getClass()) return false; + MethodKey other = (MethodKey) obj; + if (arity != other.arity) return false; + if (!name.equals(other.name)) return false; + return true; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(name); + sb.append('/'); + sb.append(arity); + return sb.toString(); + } + } public static final class Struct { public final String name; public final Class clazz; public final org.objectweb.asm.Type type; - public final Map constructors; - public final Map functions; - public final Map methods; + public final Map constructors; + public final Map staticMethods; + public final Map methods; - public final Map statics; + public final Map staticMembers; public final Map members; private Struct(final String name, final Class clazz, final org.objectweb.asm.Type type) { @@ -213,10 +269,10 @@ public final class Definition { this.type = type; constructors = new HashMap<>(); - functions = new HashMap<>(); + staticMethods = new HashMap<>(); methods = new HashMap<>(); - statics = new HashMap<>(); + staticMembers = new HashMap<>(); members = new HashMap<>(); } @@ -226,10 +282,10 @@ public final class Definition { type = struct.type; constructors = Collections.unmodifiableMap(struct.constructors); - functions = Collections.unmodifiableMap(struct.functions); + staticMethods = Collections.unmodifiableMap(struct.staticMethods); methods = Collections.unmodifiableMap(struct.methods); - statics = Collections.unmodifiableMap(struct.statics); + staticMembers = Collections.unmodifiableMap(struct.staticMembers); members = Collections.unmodifiableMap(struct.members); } @@ -304,11 +360,11 @@ public final class Definition { } public static final class RuntimeClass { - public final Map methods; + public final Map methods; public final Map getters; public final Map setters; - private RuntimeClass(final Map methods, + private RuntimeClass(final Map methods, final Map getters, final Map setters) { this.methods = methods; this.getters = getters; @@ -395,6 +451,9 @@ public final class Definition { public final Type longsType; public final Type doublesType; public final Type geoPointsType; + + // for testing features not currently "used" by the whitelist (we should not rush the API for that!) + public final Type featureTestType; private Definition() { structsMap = new HashMap<>(); @@ -476,6 +535,8 @@ public final class Definition { longsType = getType("Longs"); doublesType = getType("Doubles"); geoPointsType = getType("GeoPoints"); + + featureTestType = getType("FeatureTest"); addElements(); copyStructs(); @@ -567,6 +628,8 @@ public final class Definition { this.longsType = definition.longsType; this.doublesType = definition.doublesType; this.geoPointsType = definition.geoPointsType; + + this.featureTestType = definition.featureTestType; } private void addStructs() { @@ -643,6 +706,8 @@ public final class Definition { addStruct( "Longs" , ScriptDocValues.Longs.class); addStruct( "Doubles" , ScriptDocValues.Doubles.class); addStruct( "GeoPoints" , ScriptDocValues.GeoPoints.class); + + addStruct( "FeatureTest", FeatureTest.class); } private void addElements() { @@ -765,6 +830,7 @@ public final class Definition { addMethod("String", "compareTo", null, false, intType, new Type[] {stringType}, null, null); addMethod("String", "concat", null, false, stringType, new Type[] {stringType}, null, null); addMethod("String", "endsWith", null, false, booleanType, new Type[] {stringType}, null, null); + addMethod("String", "indexOf", null, false, intType, new Type[] {stringType}, null, null); addMethod("String", "indexOf", null, false, intType, new Type[] {stringType, intType}, null, null); addMethod("String", "isEmpty", null, false, booleanType, new Type[] {}, null, null); addMethod("String", "replace", null, false, stringType, new Type[] {charseqType, charseqType}, null, null); @@ -1112,6 +1178,16 @@ public final class Definition { new Type[] { stringType }, null, null); addMethod("GeoPoints", "geohashDistanceInMiles", null, false, doubleType, new Type[] { stringType }, null, null); + + // currently FeatureTest exposes overloaded constructor, field load store, and overloaded static methods + addConstructor("FeatureTest", "new", new Type[] {}, null); + addConstructor("FeatureTest", "new", new Type[] {intType, intType}, null); + addMethod("FeatureTest", "getX", null, false, intType, new Type[] {}, null, null); + addMethod("FeatureTest", "getY", null, false, intType, new Type[] {}, null, null); + addMethod("FeatureTest", "setX", null, false, voidType, new Type[] {intType}, null, null); + addMethod("FeatureTest", "setY", null, false, voidType, new Type[] {intType}, null, null); + addMethod("FeatureTest", "overloadedStatic", null, true, booleanType, new Type[] {}, null, null); + addMethod("FeatureTest", "overloadedStatic", null, true, booleanType, new Type[] {booleanType}, null, null); } private void copyStructs() { @@ -1165,6 +1241,8 @@ public final class Definition { copyStruct("Longs", "List", "Collection", "Object"); copyStruct("Doubles", "List", "Collection", "Object"); copyStruct("GeoPoints", "List", "Collection", "Object"); + + copyStruct("FeatureTest", "Object"); } private void addTransforms() { @@ -1482,6 +1560,8 @@ public final class Definition { addRuntimeClass(longsType.struct); addRuntimeClass(doublesType.struct); addRuntimeClass(geoPointsType.struct); + + addRuntimeClass(featureTestType.struct); } private final void addStruct(final String name, final Class clazz) { @@ -1510,20 +1590,22 @@ public final class Definition { throw new IllegalArgumentException( "Invalid constructor name [" + name + "] with the struct [" + owner.name + "]."); } + + MethodKey methodKey = new MethodKey(name, args.length); - if (owner.constructors.containsKey(name)) { + if (owner.constructors.containsKey(methodKey)) { throw new IllegalArgumentException( - "Duplicate constructor name [" + name + "] found within the struct [" + owner.name + "]."); + "Duplicate constructor [" + methodKey + "] found within the struct [" + owner.name + "]."); } - if (owner.statics.containsKey(name)) { - throw new IllegalArgumentException("Constructors and functions may not have the same name" + - " [" + name + "] within the same struct [" + owner.name + "]."); + if (owner.staticMethods.containsKey(methodKey)) { + throw new IllegalArgumentException("Constructors and static methods may not have the same signature" + + " [" + methodKey + "] within the same struct [" + owner.name + "]."); } - if (owner.methods.containsKey(name)) { - throw new IllegalArgumentException("Constructors and methods may not have the same name" + - " [" + name + "] within the same struct [" + owner.name + "]."); + if (owner.methods.containsKey(methodKey)) { + throw new IllegalArgumentException("Constructors and methods may not have the same signature" + + " [" + methodKey + "] within the same struct [" + owner.name + "]."); } final Class[] classes = new Class[args.length]; @@ -1553,7 +1635,7 @@ public final class Definition { final Constructor constructor = new Constructor(name, owner, Arrays.asList(genargs != null ? genargs : args), asm, reflect); - owner.constructors.put(name, constructor); + owner.constructors.put(methodKey, constructor); } private final void addMethod(final String struct, final String name, final String alias, final boolean statik, @@ -1566,32 +1648,34 @@ public final class Definition { } if (!name.matches("^[_a-zA-Z][_a-zA-Z0-9]*$")) { - throw new IllegalArgumentException("Invalid " + (statik ? "function" : "method") + + throw new IllegalArgumentException("Invalid " + (statik ? "static method" : "method") + " name [" + name + "] with the struct [" + owner.name + "]."); } + + MethodKey methodKey = new MethodKey(name, args.length); - if (owner.constructors.containsKey(name)) { - throw new IllegalArgumentException("Constructors and " + (statik ? "functions" : "methods") + - " may not have the same name [" + name + "] within the same struct" + + if (owner.constructors.containsKey(methodKey)) { + throw new IllegalArgumentException("Constructors and " + (statik ? "static methods" : "methods") + + " may not have the same signature [" + methodKey + "] within the same struct" + " [" + owner.name + "]."); } - if (owner.statics.containsKey(name)) { + if (owner.staticMethods.containsKey(methodKey)) { if (statik) { throw new IllegalArgumentException( - "Duplicate function name [" + name + "] found within the struct [" + owner.name + "]."); + "Duplicate static method signature [" + methodKey + "] found within the struct [" + owner.name + "]."); } else { - throw new IllegalArgumentException("Functions and methods may not have the same name" + - " [" + name + "] within the same struct [" + owner.name + "]."); + throw new IllegalArgumentException("Static methods and methods may not have the same signature" + + " [" + methodKey + "] within the same struct [" + owner.name + "]."); } } - if (owner.methods.containsKey(name)) { + if (owner.methods.containsKey(methodKey)) { if (statik) { - throw new IllegalArgumentException("Functions and methods may not have the same name" + - " [" + name + "] within the same struct [" + owner.name + "]."); + throw new IllegalArgumentException("Static methods and methods may not have the same signature" + + " [" + methodKey + "] within the same struct [" + owner.name + "]."); } else { - throw new IllegalArgumentException("Duplicate method name [" + name + "]" + + throw new IllegalArgumentException("Duplicate method signature [" + methodKey + "]" + " found within the struct [" + owner.name + "]."); } } @@ -1663,14 +1747,14 @@ public final class Definition { " within the struct [" + owner.name + "] is not linked to a static Java method."); } - owner.functions.put(name, method); + owner.staticMethods.put(methodKey, method); } else { if (java.lang.reflect.Modifier.isStatic(modifiers)) { throw new IllegalArgumentException("Method [" + name + "]" + " within the struct [" + owner.name + "] is not linked to a non-static Java method."); } - owner.methods.put(name, method); + owner.methods.put(methodKey, method); } } @@ -1688,7 +1772,7 @@ public final class Definition { " name [" + name + "] with the struct [" + owner.name + "]."); } - if (owner.statics.containsKey(name)) { + if (owner.staticMembers.containsKey(name)) { if (statik) { throw new IllegalArgumentException("Duplicate static name [" + name + "]" + " found within the struct [" + owner.name + "]."); @@ -1751,7 +1835,7 @@ public final class Definition { " within the struct [" + owner.name + "] is not linked to static Java field."); } - owner.statics.put(alias == null ? name : alias, field); + owner.staticMembers.put(alias == null ? name : alias, field); } else { if (java.lang.reflect.Modifier.isStatic(modifiers)) { throw new IllegalArgumentException("Member [" + name + "]" + @@ -1785,8 +1869,10 @@ public final class Definition { final boolean object = child.clazz.equals(Object.class) && java.lang.reflect.Modifier.isInterface(owner.clazz.getModifiers()); - for (final Method method : child.methods.values()) { - if (owner.methods.get(method.name) == null) { + for (Map.Entry kvPair : child.methods.entrySet()) { + MethodKey methodKey = kvPair.getKey(); + Method method = kvPair.getValue(); + if (owner.methods.get(methodKey) == null) { final Class clazz = object ? Object.class : owner.clazz; java.lang.reflect.Method reflect; @@ -1808,7 +1894,7 @@ public final class Definition { Arrays.toString(method.reflect.getParameterTypes()) + "."); } - owner.methods.put(method.name, + owner.methods.put(methodKey, new Method(method.name, owner, method.rtn, method.arguments, method.method, reflect, handle)); } } @@ -1866,8 +1952,11 @@ public final class Definition { Type upcast = null; Type downcast = null; + // transforms are implicitly arity of 0, unless a static method where its 1 (receiver passed) + MethodKey methodKey = new MethodKey(name, statik ? 1 : 0); + if (statik) { - method = owner.functions.get(name); + method = owner.staticMethods.get(methodKey); if (method == null) { throw new IllegalArgumentException("Transform with owner struct [" + owner.name + "]" + @@ -1905,7 +1994,7 @@ public final class Definition { } } } else { - method = owner.methods.get(name); + method = owner.methods.get(methodKey); if (method == null) { throw new IllegalArgumentException("Transform with owner struct [" + owner.name + "]" + @@ -1950,7 +2039,7 @@ public final class Definition { * Precomputes a more efficient structure for dynamic method/field access. */ private void addRuntimeClass(final Struct struct) { - final Map methods = struct.methods; + final Map methods = struct.methods; final Map getters = new HashMap<>(); final Map setters = new HashMap<>(); @@ -1961,8 +2050,8 @@ public final class Definition { } // add all getters/setters - for (final Map.Entry method : methods.entrySet()) { - final String name = method.getKey(); + for (final Map.Entry method : methods.entrySet()) { + final String name = method.getKey().name; final Method m = method.getValue(); if (m.arguments.size() == 0 && diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/FeatureTest.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/FeatureTest.java new file mode 100644 index 00000000000..bef83daaad3 --- /dev/null +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/FeatureTest.java @@ -0,0 +1,66 @@ +package org.elasticsearch.painless; + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** Currently just a dummy class for testing a few features not yet exposed by whitelist! */ +public class FeatureTest { + private int x; + private int y; + + /** empty ctor */ + public FeatureTest() { + } + + /** ctor with params */ + public FeatureTest(int x, int y) { + this.x = x; + this.y = y; + } + + /** getter for x */ + public int getX() { + return x; + } + + /** setter for x */ + public void setX(int x) { + this.x = x; + } + + /** getter for y */ + public int getY() { + return y; + } + + /** setter for y */ + public void setY(int y) { + this.y = y; + } + + /** static method that returns true */ + public static boolean overloadedStatic() { + return true; + } + + /** static method that returns what you ask it */ + public static boolean overloadedStatic(boolean whatToReturn) { + return whatToReturn; + } +} diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParser.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParser.java index 9d7ae5c628d..c8e76ee4bd9 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParser.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParser.java @@ -182,6 +182,7 @@ class PainlessParser extends Parser { return getRuleContext(DeclarationContext.class,0); } public TerminalNode SEMICOLON() { return getToken(PainlessParser.SEMICOLON, 0); } + public TerminalNode EOF() { return getToken(PainlessParser.EOF, 0); } public DeclContext(StatementContext ctx) { copyFrom(ctx); } @Override public T accept(ParseTreeVisitor visitor) { @@ -192,6 +193,7 @@ class PainlessParser extends Parser { public static class BreakContext extends StatementContext { public TerminalNode BREAK() { return getToken(PainlessParser.BREAK, 0); } public TerminalNode SEMICOLON() { return getToken(PainlessParser.SEMICOLON, 0); } + public TerminalNode EOF() { return getToken(PainlessParser.EOF, 0); } public BreakContext(StatementContext ctx) { copyFrom(ctx); } @Override public T accept(ParseTreeVisitor visitor) { @@ -205,6 +207,7 @@ class PainlessParser extends Parser { return getRuleContext(ExpressionContext.class,0); } public TerminalNode SEMICOLON() { return getToken(PainlessParser.SEMICOLON, 0); } + public TerminalNode EOF() { return getToken(PainlessParser.EOF, 0); } public ThrowContext(StatementContext ctx) { copyFrom(ctx); } @Override public T accept(ParseTreeVisitor visitor) { @@ -215,6 +218,7 @@ class PainlessParser extends Parser { public static class ContinueContext extends StatementContext { public TerminalNode CONTINUE() { return getToken(PainlessParser.CONTINUE, 0); } public TerminalNode SEMICOLON() { return getToken(PainlessParser.SEMICOLON, 0); } + public TerminalNode EOF() { return getToken(PainlessParser.EOF, 0); } public ContinueContext(StatementContext ctx) { copyFrom(ctx); } @Override public T accept(ParseTreeVisitor visitor) { @@ -275,6 +279,7 @@ class PainlessParser extends Parser { return getRuleContext(ExpressionContext.class,0); } public TerminalNode SEMICOLON() { return getToken(PainlessParser.SEMICOLON, 0); } + public TerminalNode EOF() { return getToken(PainlessParser.EOF, 0); } public ExprContext(StatementContext ctx) { copyFrom(ctx); } @Override public T accept(ParseTreeVisitor visitor) { @@ -294,6 +299,7 @@ class PainlessParser extends Parser { } public TerminalNode RP() { return getToken(PainlessParser.RP, 0); } public TerminalNode SEMICOLON() { return getToken(PainlessParser.SEMICOLON, 0); } + public TerminalNode EOF() { return getToken(PainlessParser.EOF, 0); } public DoContext(StatementContext ctx) { copyFrom(ctx); } @Override public T accept(ParseTreeVisitor visitor) { @@ -348,6 +354,7 @@ class PainlessParser extends Parser { return getRuleContext(ExpressionContext.class,0); } public TerminalNode SEMICOLON() { return getToken(PainlessParser.SEMICOLON, 0); } + public TerminalNode EOF() { return getToken(PainlessParser.EOF, 0); } public ReturnContext(StatementContext ctx) { copyFrom(ctx); } @Override public T accept(ParseTreeVisitor visitor) { @@ -362,8 +369,8 @@ class PainlessParser extends Parser { int _la; try { int _alt; - setState(134); - switch ( getInterpreter().adaptivePredict(_input,15,_ctx) ) { + setState(125); + switch ( getInterpreter().adaptivePredict(_input,8,_ctx) ) { case 1: _localctx = new IfContext(_localctx); enterOuterAlt(_localctx, 1); @@ -436,69 +443,67 @@ class PainlessParser extends Parser { expression(0); setState(79); match(RP); - setState(81); + setState(80); _la = _input.LA(1); - if (_la==SEMICOLON) { - { - setState(80); - match(SEMICOLON); - } + if ( !(_la==EOF || _la==SEMICOLON) ) { + _errHandler.recoverInline(this); + } else { + consume(); } - } break; case 4: _localctx = new ForContext(_localctx); enterOuterAlt(_localctx, 4); { - setState(83); + setState(82); match(FOR); - setState(84); + setState(83); match(LP); - setState(86); + setState(85); _la = _input.LA(1); if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LP) | (1L << NEW) | (1L << BOOLNOT) | (1L << BWNOT) | (1L << ADD) | (1L << SUB) | (1L << INCR) | (1L << DECR) | (1L << OCTAL))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (HEX - 64)) | (1L << (INTEGER - 64)) | (1L << (DECIMAL - 64)) | (1L << (STRING - 64)) | (1L << (TRUE - 64)) | (1L << (FALSE - 64)) | (1L << (NULL - 64)) | (1L << (ID - 64)))) != 0)) { { - setState(85); + setState(84); initializer(); } } - setState(88); + setState(87); match(SEMICOLON); - setState(90); + setState(89); _la = _input.LA(1); if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LP) | (1L << NEW) | (1L << BOOLNOT) | (1L << BWNOT) | (1L << ADD) | (1L << SUB) | (1L << INCR) | (1L << DECR) | (1L << OCTAL))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (HEX - 64)) | (1L << (INTEGER - 64)) | (1L << (DECIMAL - 64)) | (1L << (STRING - 64)) | (1L << (TRUE - 64)) | (1L << (FALSE - 64)) | (1L << (NULL - 64)) | (1L << (ID - 64)))) != 0)) { { - setState(89); + setState(88); expression(0); } } - setState(92); + setState(91); match(SEMICOLON); - setState(94); + setState(93); _la = _input.LA(1); if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LP) | (1L << NEW) | (1L << BOOLNOT) | (1L << BWNOT) | (1L << ADD) | (1L << SUB) | (1L << INCR) | (1L << DECR) | (1L << OCTAL))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (HEX - 64)) | (1L << (INTEGER - 64)) | (1L << (DECIMAL - 64)) | (1L << (STRING - 64)) | (1L << (TRUE - 64)) | (1L << (FALSE - 64)) | (1L << (NULL - 64)) | (1L << (ID - 64)))) != 0)) { { - setState(93); + setState(92); afterthought(); } } - setState(96); + setState(95); match(RP); - setState(99); - switch ( getInterpreter().adaptivePredict(_input,7,_ctx) ) { + setState(98); + switch ( getInterpreter().adaptivePredict(_input,6,_ctx) ) { case 1: { - setState(97); + setState(96); block(); } break; case 2: { - setState(98); + setState(97); empty(); } break; @@ -509,81 +514,73 @@ class PainlessParser extends Parser { _localctx = new DeclContext(_localctx); enterOuterAlt(_localctx, 5); { - setState(101); + setState(100); declaration(); - setState(103); + setState(101); _la = _input.LA(1); - if (_la==SEMICOLON) { - { - setState(102); - match(SEMICOLON); - } + if ( !(_la==EOF || _la==SEMICOLON) ) { + _errHandler.recoverInline(this); + } else { + consume(); } - } break; case 6: _localctx = new ContinueContext(_localctx); enterOuterAlt(_localctx, 6); { - setState(105); + setState(103); match(CONTINUE); - setState(107); + setState(104); _la = _input.LA(1); - if (_la==SEMICOLON) { - { - setState(106); - match(SEMICOLON); - } + if ( !(_la==EOF || _la==SEMICOLON) ) { + _errHandler.recoverInline(this); + } else { + consume(); } - } break; case 7: _localctx = new BreakContext(_localctx); enterOuterAlt(_localctx, 7); { - setState(109); + setState(105); match(BREAK); - setState(111); + setState(106); _la = _input.LA(1); - if (_la==SEMICOLON) { - { - setState(110); - match(SEMICOLON); - } + if ( !(_la==EOF || _la==SEMICOLON) ) { + _errHandler.recoverInline(this); + } else { + consume(); } - } break; case 8: _localctx = new ReturnContext(_localctx); enterOuterAlt(_localctx, 8); { - setState(113); + setState(107); match(RETURN); - setState(114); + setState(108); expression(0); - setState(116); + setState(109); _la = _input.LA(1); - if (_la==SEMICOLON) { - { - setState(115); - match(SEMICOLON); - } + if ( !(_la==EOF || _la==SEMICOLON) ) { + _errHandler.recoverInline(this); + } else { + consume(); } - } break; case 9: _localctx = new TryContext(_localctx); enterOuterAlt(_localctx, 9); { - setState(118); + setState(111); match(TRY); - setState(119); + setState(112); block(); - setState(121); + setState(114); _errHandler.sync(this); _alt = 1; do { @@ -591,7 +588,7 @@ class PainlessParser extends Parser { case 1: { { - setState(120); + setState(113); trap(); } } @@ -599,9 +596,9 @@ class PainlessParser extends Parser { default: throw new NoViableAltException(this); } - setState(123); + setState(116); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,12,_ctx); + _alt = getInterpreter().adaptivePredict(_input,7,_ctx); } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); } break; @@ -609,36 +606,32 @@ class PainlessParser extends Parser { _localctx = new ThrowContext(_localctx); enterOuterAlt(_localctx, 10); { - setState(125); + setState(118); match(THROW); - setState(126); + setState(119); expression(0); - setState(128); + setState(120); _la = _input.LA(1); - if (_la==SEMICOLON) { - { - setState(127); - match(SEMICOLON); - } + if ( !(_la==EOF || _la==SEMICOLON) ) { + _errHandler.recoverInline(this); + } else { + consume(); } - } break; case 11: _localctx = new ExprContext(_localctx); enterOuterAlt(_localctx, 11); { - setState(130); + setState(122); expression(0); - setState(132); + setState(123); _la = _input.LA(1); - if (_la==SEMICOLON) { - { - setState(131); - match(SEMICOLON); - } + if ( !(_la==EOF || _la==SEMICOLON) ) { + _errHandler.recoverInline(this); + } else { + consume(); } - } break; } @@ -698,29 +691,29 @@ class PainlessParser extends Parser { enterRule(_localctx, 4, RULE_block); int _la; try { - setState(145); + setState(136); switch (_input.LA(1)) { case LBRACK: _localctx = new MultipleContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(136); + setState(127); match(LBRACK); - setState(138); + setState(129); _errHandler.sync(this); _la = _input.LA(1); do { { { - setState(137); + setState(128); statement(); } } - setState(140); + setState(131); _errHandler.sync(this); _la = _input.LA(1); } while ( (((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LP) | (1L << IF) | (1L << WHILE) | (1L << DO) | (1L << FOR) | (1L << CONTINUE) | (1L << BREAK) | (1L << RETURN) | (1L << NEW) | (1L << TRY) | (1L << THROW) | (1L << BOOLNOT) | (1L << BWNOT) | (1L << ADD) | (1L << SUB) | (1L << INCR) | (1L << DECR) | (1L << OCTAL))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (HEX - 64)) | (1L << (INTEGER - 64)) | (1L << (DECIMAL - 64)) | (1L << (STRING - 64)) | (1L << (TRUE - 64)) | (1L << (FALSE - 64)) | (1L << (NULL - 64)) | (1L << (ID - 64)))) != 0) ); - setState(142); + setState(133); match(RBRACK); } break; @@ -753,7 +746,7 @@ class PainlessParser extends Parser { _localctx = new SingleContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(144); + setState(135); statement(); } break; @@ -792,19 +785,19 @@ class PainlessParser extends Parser { EmptyContext _localctx = new EmptyContext(_ctx, getState()); enterRule(_localctx, 6, RULE_empty); try { - setState(149); + setState(140); switch (_input.LA(1)) { case LBRACK: enterOuterAlt(_localctx, 1); { - setState(147); + setState(138); emptyscope(); } break; case SEMICOLON: enterOuterAlt(_localctx, 2); { - setState(148); + setState(139); match(SEMICOLON); } break; @@ -843,9 +836,9 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(151); + setState(142); match(LBRACK); - setState(152); + setState(143); match(RBRACK); } } @@ -882,19 +875,19 @@ class PainlessParser extends Parser { InitializerContext _localctx = new InitializerContext(_ctx, getState()); enterRule(_localctx, 10, RULE_initializer); try { - setState(156); - switch ( getInterpreter().adaptivePredict(_input,19,_ctx) ) { + setState(147); + switch ( getInterpreter().adaptivePredict(_input,12,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(154); + setState(145); declaration(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(155); + setState(146); expression(0); } break; @@ -932,7 +925,7 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(158); + setState(149); expression(0); } } @@ -979,23 +972,23 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(160); + setState(151); decltype(); - setState(161); + setState(152); declvar(); - setState(166); + setState(157); _errHandler.sync(this); _la = _input.LA(1); while (_la==COMMA) { { { - setState(162); + setState(153); match(COMMA); - setState(163); + setState(154); declvar(); } } - setState(168); + setState(159); _errHandler.sync(this); _la = _input.LA(1); } @@ -1042,21 +1035,21 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(169); + setState(160); identifier(); - setState(174); + setState(165); _errHandler.sync(this); _la = _input.LA(1); while (_la==LBRACE) { { { - setState(170); + setState(161); match(LBRACE); - setState(171); + setState(162); match(RBRACE); } } - setState(176); + setState(167); _errHandler.sync(this); _la = _input.LA(1); } @@ -1099,15 +1092,15 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(177); + setState(168); identifier(); - setState(180); + setState(171); _la = _input.LA(1); if (_la==ASSIGN) { { - setState(178); + setState(169); match(ASSIGN); - setState(179); + setState(170); expression(0); } } @@ -1158,29 +1151,29 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(182); + setState(173); match(CATCH); - setState(183); + setState(174); match(LP); { - setState(184); + setState(175); identifier(); - setState(185); + setState(176); identifier(); } - setState(187); + setState(178); match(RP); - setState(190); - switch ( getInterpreter().adaptivePredict(_input,23,_ctx) ) { + setState(181); + switch ( getInterpreter().adaptivePredict(_input,16,_ctx) ) { case 1: { - setState(188); + setState(179); block(); } break; case 2: { - setState(189); + setState(180); emptyscope(); } break; @@ -1220,13 +1213,13 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(192); + setState(183); match(ID); - setState(194); - switch ( getInterpreter().adaptivePredict(_input,24,_ctx) ) { + setState(185); + switch ( getInterpreter().adaptivePredict(_input,17,_ctx) ) { case 1: { - setState(193); + setState(184); generic(); } break; @@ -1275,27 +1268,27 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(196); + setState(187); match(LT); - setState(197); + setState(188); identifier(); - setState(202); + setState(193); _errHandler.sync(this); _la = _input.LA(1); while (_la==COMMA) { { { - setState(198); + setState(189); match(COMMA); - setState(199); + setState(190); identifier(); } } - setState(204); + setState(195); _errHandler.sync(this); _la = _input.LA(1); } - setState(205); + setState(196); match(GT); } } @@ -1563,22 +1556,22 @@ class PainlessParser extends Parser { int _alt; enterOuterAlt(_localctx, 1); { - setState(233); - switch ( getInterpreter().adaptivePredict(_input,26,_ctx) ) { + setState(224); + switch ( getInterpreter().adaptivePredict(_input,19,_ctx) ) { case 1: { _localctx = new UnaryContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(208); + setState(199); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << BOOLNOT) | (1L << BWNOT) | (1L << ADD) | (1L << SUB))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(209); + setState(200); expression(14); } break; @@ -1587,13 +1580,13 @@ class PainlessParser extends Parser { _localctx = new CastContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(210); + setState(201); match(LP); - setState(211); + setState(202); decltype(); - setState(212); + setState(203); match(RP); - setState(213); + setState(204); expression(13); } break; @@ -1602,16 +1595,16 @@ class PainlessParser extends Parser { _localctx = new AssignmentContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(215); + setState(206); chain(); - setState(216); + setState(207); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ASSIGN) | (1L << AADD) | (1L << ASUB) | (1L << AMUL) | (1L << ADIV) | (1L << AREM) | (1L << AAND) | (1L << AXOR) | (1L << AOR) | (1L << ALSH) | (1L << ARSH) | (1L << AUSH))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(217); + setState(208); expression(1); } break; @@ -1620,11 +1613,11 @@ class PainlessParser extends Parser { _localctx = new PrecedenceContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(219); + setState(210); match(LP); - setState(220); + setState(211); expression(0); - setState(221); + setState(212); match(RP); } break; @@ -1633,7 +1626,7 @@ class PainlessParser extends Parser { _localctx = new NumericContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(223); + setState(214); _la = _input.LA(1); if ( !(((((_la - 63)) & ~0x3f) == 0 && ((1L << (_la - 63)) & ((1L << (OCTAL - 63)) | (1L << (HEX - 63)) | (1L << (INTEGER - 63)) | (1L << (DECIMAL - 63)))) != 0)) ) { _errHandler.recoverInline(this); @@ -1647,7 +1640,7 @@ class PainlessParser extends Parser { _localctx = new TrueContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(224); + setState(215); match(TRUE); } break; @@ -1656,7 +1649,7 @@ class PainlessParser extends Parser { _localctx = new FalseContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(225); + setState(216); match(FALSE); } break; @@ -1665,7 +1658,7 @@ class PainlessParser extends Parser { _localctx = new NullContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(226); + setState(217); match(NULL); } break; @@ -1674,9 +1667,9 @@ class PainlessParser extends Parser { _localctx = new PostincContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(227); + setState(218); chain(); - setState(228); + setState(219); _la = _input.LA(1); if ( !(_la==INCR || _la==DECR) ) { _errHandler.recoverInline(this); @@ -1690,14 +1683,14 @@ class PainlessParser extends Parser { _localctx = new PreincContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(230); + setState(221); _la = _input.LA(1); if ( !(_la==INCR || _la==DECR) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(231); + setState(222); chain(); } break; @@ -1706,36 +1699,36 @@ class PainlessParser extends Parser { _localctx = new ReadContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(232); + setState(223); chain(); } break; } _ctx.stop = _input.LT(-1); - setState(273); + setState(264); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,28,_ctx); + _alt = getInterpreter().adaptivePredict(_input,21,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { if ( _parseListeners!=null ) triggerExitRuleEvent(); _prevctx = _localctx; { - setState(271); - switch ( getInterpreter().adaptivePredict(_input,27,_ctx) ) { + setState(262); + switch ( getInterpreter().adaptivePredict(_input,20,_ctx) ) { case 1: { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(235); + setState(226); if (!(precpred(_ctx, 12))) throw new FailedPredicateException(this, "precpred(_ctx, 12)"); - setState(236); + setState(227); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << MUL) | (1L << DIV) | (1L << REM))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(237); + setState(228); expression(13); } break; @@ -1743,16 +1736,16 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(238); + setState(229); if (!(precpred(_ctx, 11))) throw new FailedPredicateException(this, "precpred(_ctx, 11)"); - setState(239); + setState(230); _la = _input.LA(1); if ( !(_la==ADD || _la==SUB) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(240); + setState(231); expression(12); } break; @@ -1760,16 +1753,16 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(241); + setState(232); if (!(precpred(_ctx, 10))) throw new FailedPredicateException(this, "precpred(_ctx, 10)"); - setState(242); + setState(233); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LSH) | (1L << RSH) | (1L << USH))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(243); + setState(234); expression(11); } break; @@ -1777,16 +1770,16 @@ class PainlessParser extends Parser { { _localctx = new CompContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(244); + setState(235); if (!(precpred(_ctx, 9))) throw new FailedPredicateException(this, "precpred(_ctx, 9)"); - setState(245); + setState(236); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LT) | (1L << LTE) | (1L << GT) | (1L << GTE))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(246); + setState(237); expression(10); } break; @@ -1794,16 +1787,16 @@ class PainlessParser extends Parser { { _localctx = new CompContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(247); + setState(238); if (!(precpred(_ctx, 8))) throw new FailedPredicateException(this, "precpred(_ctx, 8)"); - setState(248); + setState(239); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << EQ) | (1L << EQR) | (1L << NE) | (1L << NER))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(249); + setState(240); expression(9); } break; @@ -1811,11 +1804,11 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(250); + setState(241); if (!(precpred(_ctx, 7))) throw new FailedPredicateException(this, "precpred(_ctx, 7)"); - setState(251); + setState(242); match(BWAND); - setState(252); + setState(243); expression(8); } break; @@ -1823,11 +1816,11 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(253); + setState(244); if (!(precpred(_ctx, 6))) throw new FailedPredicateException(this, "precpred(_ctx, 6)"); - setState(254); + setState(245); match(XOR); - setState(255); + setState(246); expression(7); } break; @@ -1835,11 +1828,11 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(256); + setState(247); if (!(precpred(_ctx, 5))) throw new FailedPredicateException(this, "precpred(_ctx, 5)"); - setState(257); + setState(248); match(BWOR); - setState(258); + setState(249); expression(6); } break; @@ -1847,11 +1840,11 @@ class PainlessParser extends Parser { { _localctx = new BoolContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(259); + setState(250); if (!(precpred(_ctx, 4))) throw new FailedPredicateException(this, "precpred(_ctx, 4)"); - setState(260); + setState(251); match(BOOLAND); - setState(261); + setState(252); expression(5); } break; @@ -1859,11 +1852,11 @@ class PainlessParser extends Parser { { _localctx = new BoolContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(262); + setState(253); if (!(precpred(_ctx, 3))) throw new FailedPredicateException(this, "precpred(_ctx, 3)"); - setState(263); + setState(254); match(BOOLOR); - setState(264); + setState(255); expression(4); } break; @@ -1871,24 +1864,24 @@ class PainlessParser extends Parser { { _localctx = new ConditionalContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(265); + setState(256); if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); - setState(266); + setState(257); match(COND); - setState(267); + setState(258); expression(0); - setState(268); + setState(259); match(COLON); - setState(269); + setState(260); expression(2); } break; } } } - setState(275); + setState(266); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,28,_ctx); + _alt = getInterpreter().adaptivePredict(_input,21,_ctx); } } } @@ -1934,40 +1927,40 @@ class PainlessParser extends Parser { ChainContext _localctx = new ChainContext(_ctx, getState()); enterRule(_localctx, 28, RULE_chain); try { - setState(281); - switch ( getInterpreter().adaptivePredict(_input,29,_ctx) ) { + setState(272); + switch ( getInterpreter().adaptivePredict(_input,22,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(276); + setState(267); linkprec(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(277); + setState(268); linkcast(); } break; case 3: enterOuterAlt(_localctx, 3); { - setState(278); + setState(269); linkvar(); } break; case 4: enterOuterAlt(_localctx, 4); { - setState(279); + setState(270); linknew(); } break; case 5: enterOuterAlt(_localctx, 5); { - setState(280); + setState(271); linkstring(); } break; @@ -2025,54 +2018,54 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(283); + setState(274); match(LP); - setState(289); - switch ( getInterpreter().adaptivePredict(_input,30,_ctx) ) { + setState(280); + switch ( getInterpreter().adaptivePredict(_input,23,_ctx) ) { case 1: { - setState(284); + setState(275); linkprec(); } break; case 2: { - setState(285); + setState(276); linkcast(); } break; case 3: { - setState(286); + setState(277); linkvar(); } break; case 4: { - setState(287); + setState(278); linknew(); } break; case 5: { - setState(288); + setState(279); linkstring(); } break; } - setState(291); + setState(282); match(RP); - setState(294); - switch ( getInterpreter().adaptivePredict(_input,31,_ctx) ) { + setState(285); + switch ( getInterpreter().adaptivePredict(_input,24,_ctx) ) { case 1: { - setState(292); + setState(283); linkdot(); } break; case 2: { - setState(293); + setState(284); linkbrace(); } break; @@ -2128,41 +2121,41 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(296); + setState(287); match(LP); - setState(297); + setState(288); decltype(); - setState(298); + setState(289); match(RP); - setState(304); - switch ( getInterpreter().adaptivePredict(_input,32,_ctx) ) { + setState(295); + switch ( getInterpreter().adaptivePredict(_input,25,_ctx) ) { case 1: { - setState(299); + setState(290); linkprec(); } break; case 2: { - setState(300); + setState(291); linkcast(); } break; case 3: { - setState(301); + setState(292); linkvar(); } break; case 4: { - setState(302); + setState(293); linknew(); } break; case 5: { - setState(303); + setState(294); linkstring(); } break; @@ -2209,23 +2202,23 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(306); + setState(297); match(LBRACE); - setState(307); + setState(298); expression(0); - setState(308); + setState(299); match(RBRACE); - setState(311); - switch ( getInterpreter().adaptivePredict(_input,33,_ctx) ) { + setState(302); + switch ( getInterpreter().adaptivePredict(_input,26,_ctx) ) { case 1: { - setState(309); + setState(300); linkdot(); } break; case 2: { - setState(310); + setState(301); linkbrace(); } break; @@ -2268,19 +2261,19 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(313); + setState(304); match(DOT); - setState(316); - switch ( getInterpreter().adaptivePredict(_input,34,_ctx) ) { + setState(307); + switch ( getInterpreter().adaptivePredict(_input,27,_ctx) ) { case 1: { - setState(314); + setState(305); linkcall(); } break; case 2: { - setState(315); + setState(306); linkfield(); } break; @@ -2326,21 +2319,21 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(318); + setState(309); match(EXTID); - setState(319); + setState(310); arguments(); - setState(322); - switch ( getInterpreter().adaptivePredict(_input,35,_ctx) ) { + setState(313); + switch ( getInterpreter().adaptivePredict(_input,28,_ctx) ) { case 1: { - setState(320); + setState(311); linkdot(); } break; case 2: { - setState(321); + setState(312); linkbrace(); } break; @@ -2385,19 +2378,19 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(324); + setState(315); identifier(); - setState(327); - switch ( getInterpreter().adaptivePredict(_input,36,_ctx) ) { + setState(318); + switch ( getInterpreter().adaptivePredict(_input,29,_ctx) ) { case 1: { - setState(325); + setState(316); linkdot(); } break; case 2: { - setState(326); + setState(317); linkbrace(); } break; @@ -2442,24 +2435,24 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(329); + setState(320); _la = _input.LA(1); if ( !(_la==EXTINTEGER || _la==EXTID) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(332); - switch ( getInterpreter().adaptivePredict(_input,37,_ctx) ) { + setState(323); + switch ( getInterpreter().adaptivePredict(_input,30,_ctx) ) { case 1: { - setState(330); + setState(321); linkdot(); } break; case 2: { - setState(331); + setState(322); linkbrace(); } break; @@ -2520,22 +2513,22 @@ class PainlessParser extends Parser { int _alt; enterOuterAlt(_localctx, 1); { - setState(334); + setState(325); match(NEW); - setState(335); + setState(326); identifier(); - setState(351); + setState(342); switch (_input.LA(1)) { case LP: { { - setState(336); + setState(327); arguments(); - setState(338); - switch ( getInterpreter().adaptivePredict(_input,38,_ctx) ) { + setState(329); + switch ( getInterpreter().adaptivePredict(_input,31,_ctx) ) { case 1: { - setState(337); + setState(328); linkdot(); } break; @@ -2546,7 +2539,7 @@ class PainlessParser extends Parser { case LBRACE: { { - setState(344); + setState(335); _errHandler.sync(this); _alt = 1; do { @@ -2554,11 +2547,11 @@ class PainlessParser extends Parser { case 1: { { - setState(340); + setState(331); match(LBRACE); - setState(341); + setState(332); expression(0); - setState(342); + setState(333); match(RBRACE); } } @@ -2566,15 +2559,15 @@ class PainlessParser extends Parser { default: throw new NoViableAltException(this); } - setState(346); + setState(337); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,39,_ctx); + _alt = getInterpreter().adaptivePredict(_input,32,_ctx); } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); - setState(349); - switch ( getInterpreter().adaptivePredict(_input,40,_ctx) ) { + setState(340); + switch ( getInterpreter().adaptivePredict(_input,33,_ctx) ) { case 1: { - setState(348); + setState(339); linkdot(); } break; @@ -2623,19 +2616,19 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(353); + setState(344); match(STRING); - setState(356); - switch ( getInterpreter().adaptivePredict(_input,42,_ctx) ) { + setState(347); + switch ( getInterpreter().adaptivePredict(_input,35,_ctx) ) { case 1: { - setState(354); + setState(345); linkdot(); } break; case 2: { - setState(355); + setState(346); linkbrace(); } break; @@ -2685,34 +2678,34 @@ class PainlessParser extends Parser { enterOuterAlt(_localctx, 1); { { - setState(358); + setState(349); match(LP); - setState(367); + setState(358); _la = _input.LA(1); if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LP) | (1L << NEW) | (1L << BOOLNOT) | (1L << BWNOT) | (1L << ADD) | (1L << SUB) | (1L << INCR) | (1L << DECR) | (1L << OCTAL))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (HEX - 64)) | (1L << (INTEGER - 64)) | (1L << (DECIMAL - 64)) | (1L << (STRING - 64)) | (1L << (TRUE - 64)) | (1L << (FALSE - 64)) | (1L << (NULL - 64)) | (1L << (ID - 64)))) != 0)) { { - setState(359); + setState(350); expression(0); - setState(364); + setState(355); _errHandler.sync(this); _la = _input.LA(1); while (_la==COMMA) { { { - setState(360); + setState(351); match(COMMA); - setState(361); + setState(352); expression(0); } } - setState(366); + setState(357); _errHandler.sync(this); _la = _input.LA(1); } } } - setState(369); + setState(360); match(RP); } } @@ -2764,150 +2757,145 @@ class PainlessParser extends Parser { } public static final String _serializedATN = - "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3K\u0176\4\2\t\2\4"+ + "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3K\u016d\4\2\t\2\4"+ "\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13\t"+ "\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22\t\22"+ "\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31\t\31"+ "\4\32\t\32\3\2\6\2\66\n\2\r\2\16\2\67\3\2\3\2\3\3\3\3\3\3\3\3\3\3\3\3"+ "\3\3\5\3C\n\3\3\3\3\3\3\3\3\3\3\3\3\3\5\3K\n\3\3\3\3\3\3\3\3\3\3\3\3\3"+ - "\3\3\5\3T\n\3\3\3\3\3\3\3\5\3Y\n\3\3\3\3\3\5\3]\n\3\3\3\3\3\5\3a\n\3\3"+ - "\3\3\3\3\3\5\3f\n\3\3\3\3\3\5\3j\n\3\3\3\3\3\5\3n\n\3\3\3\3\3\5\3r\n\3"+ - "\3\3\3\3\3\3\5\3w\n\3\3\3\3\3\3\3\6\3|\n\3\r\3\16\3}\3\3\3\3\3\3\5\3\u0083"+ - "\n\3\3\3\3\3\5\3\u0087\n\3\5\3\u0089\n\3\3\4\3\4\6\4\u008d\n\4\r\4\16"+ - "\4\u008e\3\4\3\4\3\4\5\4\u0094\n\4\3\5\3\5\5\5\u0098\n\5\3\6\3\6\3\6\3"+ - "\7\3\7\5\7\u009f\n\7\3\b\3\b\3\t\3\t\3\t\3\t\7\t\u00a7\n\t\f\t\16\t\u00aa"+ - "\13\t\3\n\3\n\3\n\7\n\u00af\n\n\f\n\16\n\u00b2\13\n\3\13\3\13\3\13\5\13"+ - "\u00b7\n\13\3\f\3\f\3\f\3\f\3\f\3\f\3\f\3\f\5\f\u00c1\n\f\3\r\3\r\5\r"+ - "\u00c5\n\r\3\16\3\16\3\16\3\16\7\16\u00cb\n\16\f\16\16\16\u00ce\13\16"+ - "\3\16\3\16\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17"+ + "\3\3\3\3\3\3\3\3\3\3\5\3X\n\3\3\3\3\3\5\3\\\n\3\3\3\3\3\5\3`\n\3\3\3\3"+ + "\3\3\3\5\3e\n\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3"+ + "\3\6\3u\n\3\r\3\16\3v\3\3\3\3\3\3\3\3\3\3\3\3\3\3\5\3\u0080\n\3\3\4\3"+ + "\4\6\4\u0084\n\4\r\4\16\4\u0085\3\4\3\4\3\4\5\4\u008b\n\4\3\5\3\5\5\5"+ + "\u008f\n\5\3\6\3\6\3\6\3\7\3\7\5\7\u0096\n\7\3\b\3\b\3\t\3\t\3\t\3\t\7"+ + "\t\u009e\n\t\f\t\16\t\u00a1\13\t\3\n\3\n\3\n\7\n\u00a6\n\n\f\n\16\n\u00a9"+ + "\13\n\3\13\3\13\3\13\5\13\u00ae\n\13\3\f\3\f\3\f\3\f\3\f\3\f\3\f\3\f\5"+ + "\f\u00b8\n\f\3\r\3\r\5\r\u00bc\n\r\3\16\3\16\3\16\3\16\7\16\u00c2\n\16"+ + "\f\16\16\16\u00c5\13\16\3\16\3\16\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3"+ + "\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3"+ + "\17\3\17\3\17\3\17\3\17\5\17\u00e3\n\17\3\17\3\17\3\17\3\17\3\17\3\17"+ "\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17"+ - "\5\17\u00ec\n\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17"+ "\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17"+ - "\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\7\17\u0112\n\17"+ - "\f\17\16\17\u0115\13\17\3\20\3\20\3\20\3\20\3\20\5\20\u011c\n\20\3\21"+ - "\3\21\3\21\3\21\3\21\3\21\5\21\u0124\n\21\3\21\3\21\3\21\5\21\u0129\n"+ - "\21\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\5\22\u0133\n\22\3\23\3\23"+ - "\3\23\3\23\3\23\5\23\u013a\n\23\3\24\3\24\3\24\5\24\u013f\n\24\3\25\3"+ - "\25\3\25\3\25\5\25\u0145\n\25\3\26\3\26\3\26\5\26\u014a\n\26\3\27\3\27"+ - "\3\27\5\27\u014f\n\27\3\30\3\30\3\30\3\30\5\30\u0155\n\30\3\30\3\30\3"+ - "\30\3\30\6\30\u015b\n\30\r\30\16\30\u015c\3\30\5\30\u0160\n\30\5\30\u0162"+ - "\n\30\3\31\3\31\3\31\5\31\u0167\n\31\3\32\3\32\3\32\3\32\7\32\u016d\n"+ - "\32\f\32\16\32\u0170\13\32\5\32\u0172\n\32\3\32\3\32\3\32\2\3\34\33\2"+ - "\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$&(*,.\60\62\2\f\4\2\32\33\37"+ - " \3\2\65@\3\2AD\3\2\63\64\3\2\34\36\3\2\37 \3\2!#\3\2$\'\3\2(+\3\2JK\u01b3"+ - "\2\65\3\2\2\2\4\u0088\3\2\2\2\6\u0093\3\2\2\2\b\u0097\3\2\2\2\n\u0099"+ - "\3\2\2\2\f\u009e\3\2\2\2\16\u00a0\3\2\2\2\20\u00a2\3\2\2\2\22\u00ab\3"+ - "\2\2\2\24\u00b3\3\2\2\2\26\u00b8\3\2\2\2\30\u00c2\3\2\2\2\32\u00c6\3\2"+ - "\2\2\34\u00eb\3\2\2\2\36\u011b\3\2\2\2 \u011d\3\2\2\2\"\u012a\3\2\2\2"+ - "$\u0134\3\2\2\2&\u013b\3\2\2\2(\u0140\3\2\2\2*\u0146\3\2\2\2,\u014b\3"+ - "\2\2\2.\u0150\3\2\2\2\60\u0163\3\2\2\2\62\u0168\3\2\2\2\64\66\5\4\3\2"+ - "\65\64\3\2\2\2\66\67\3\2\2\2\67\65\3\2\2\2\678\3\2\2\289\3\2\2\29:\7\2"+ - "\2\3:\3\3\2\2\2;<\7\16\2\2<=\7\t\2\2=>\5\34\17\2>?\7\n\2\2?B\5\6\4\2@"+ - "A\7\17\2\2AC\5\6\4\2B@\3\2\2\2BC\3\2\2\2C\u0089\3\2\2\2DE\7\20\2\2EF\7"+ - "\t\2\2FG\5\34\17\2GJ\7\n\2\2HK\5\6\4\2IK\5\b\5\2JH\3\2\2\2JI\3\2\2\2K"+ - "\u0089\3\2\2\2LM\7\21\2\2MN\5\6\4\2NO\7\20\2\2OP\7\t\2\2PQ\5\34\17\2Q"+ - "S\7\n\2\2RT\7\r\2\2SR\3\2\2\2ST\3\2\2\2T\u0089\3\2\2\2UV\7\22\2\2VX\7"+ - "\t\2\2WY\5\f\7\2XW\3\2\2\2XY\3\2\2\2YZ\3\2\2\2Z\\\7\r\2\2[]\5\34\17\2"+ - "\\[\3\2\2\2\\]\3\2\2\2]^\3\2\2\2^`\7\r\2\2_a\5\16\b\2`_\3\2\2\2`a\3\2"+ - "\2\2ab\3\2\2\2be\7\n\2\2cf\5\6\4\2df\5\b\5\2ec\3\2\2\2ed\3\2\2\2f\u0089"+ - "\3\2\2\2gi\5\20\t\2hj\7\r\2\2ih\3\2\2\2ij\3\2\2\2j\u0089\3\2\2\2km\7\23"+ - "\2\2ln\7\r\2\2ml\3\2\2\2mn\3\2\2\2n\u0089\3\2\2\2oq\7\24\2\2pr\7\r\2\2"+ - "qp\3\2\2\2qr\3\2\2\2r\u0089\3\2\2\2st\7\25\2\2tv\5\34\17\2uw\7\r\2\2v"+ - "u\3\2\2\2vw\3\2\2\2w\u0089\3\2\2\2xy\7\27\2\2y{\5\6\4\2z|\5\26\f\2{z\3"+ - "\2\2\2|}\3\2\2\2}{\3\2\2\2}~\3\2\2\2~\u0089\3\2\2\2\177\u0080\7\31\2\2"+ - "\u0080\u0082\5\34\17\2\u0081\u0083\7\r\2\2\u0082\u0081\3\2\2\2\u0082\u0083"+ - "\3\2\2\2\u0083\u0089\3\2\2\2\u0084\u0086\5\34\17\2\u0085\u0087\7\r\2\2"+ - "\u0086\u0085\3\2\2\2\u0086\u0087\3\2\2\2\u0087\u0089\3\2\2\2\u0088;\3"+ - "\2\2\2\u0088D\3\2\2\2\u0088L\3\2\2\2\u0088U\3\2\2\2\u0088g\3\2\2\2\u0088"+ - "k\3\2\2\2\u0088o\3\2\2\2\u0088s\3\2\2\2\u0088x\3\2\2\2\u0088\177\3\2\2"+ - "\2\u0088\u0084\3\2\2\2\u0089\5\3\2\2\2\u008a\u008c\7\5\2\2\u008b\u008d"+ - "\5\4\3\2\u008c\u008b\3\2\2\2\u008d\u008e\3\2\2\2\u008e\u008c\3\2\2\2\u008e"+ - "\u008f\3\2\2\2\u008f\u0090\3\2\2\2\u0090\u0091\7\6\2\2\u0091\u0094\3\2"+ - "\2\2\u0092\u0094\5\4\3\2\u0093\u008a\3\2\2\2\u0093\u0092\3\2\2\2\u0094"+ - "\7\3\2\2\2\u0095\u0098\5\n\6\2\u0096\u0098\7\r\2\2\u0097\u0095\3\2\2\2"+ - "\u0097\u0096\3\2\2\2\u0098\t\3\2\2\2\u0099\u009a\7\5\2\2\u009a\u009b\7"+ - "\6\2\2\u009b\13\3\2\2\2\u009c\u009f\5\20\t\2\u009d\u009f\5\34\17\2\u009e"+ - "\u009c\3\2\2\2\u009e\u009d\3\2\2\2\u009f\r\3\2\2\2\u00a0\u00a1\5\34\17"+ - "\2\u00a1\17\3\2\2\2\u00a2\u00a3\5\22\n\2\u00a3\u00a8\5\24\13\2\u00a4\u00a5"+ - "\7\f\2\2\u00a5\u00a7\5\24\13\2\u00a6\u00a4\3\2\2\2\u00a7\u00aa\3\2\2\2"+ - "\u00a8\u00a6\3\2\2\2\u00a8\u00a9\3\2\2\2\u00a9\21\3\2\2\2\u00aa\u00a8"+ - "\3\2\2\2\u00ab\u00b0\5\30\r\2\u00ac\u00ad\7\7\2\2\u00ad\u00af\7\b\2\2"+ - "\u00ae\u00ac\3\2\2\2\u00af\u00b2\3\2\2\2\u00b0\u00ae\3\2\2\2\u00b0\u00b1"+ - "\3\2\2\2\u00b1\23\3\2\2\2\u00b2\u00b0\3\2\2\2\u00b3\u00b6\5\30\r\2\u00b4"+ - "\u00b5\7\65\2\2\u00b5\u00b7\5\34\17\2\u00b6\u00b4\3\2\2\2\u00b6\u00b7"+ - "\3\2\2\2\u00b7\25\3\2\2\2\u00b8\u00b9\7\30\2\2\u00b9\u00ba\7\t\2\2\u00ba"+ - "\u00bb\5\30\r\2\u00bb\u00bc\5\30\r\2\u00bc\u00bd\3\2\2\2\u00bd\u00c0\7"+ - "\n\2\2\u00be\u00c1\5\6\4\2\u00bf\u00c1\5\n\6\2\u00c0\u00be\3\2\2\2\u00c0"+ - "\u00bf\3\2\2\2\u00c1\27\3\2\2\2\u00c2\u00c4\7I\2\2\u00c3\u00c5\5\32\16"+ - "\2\u00c4\u00c3\3\2\2\2\u00c4\u00c5\3\2\2\2\u00c5\31\3\2\2\2\u00c6\u00c7"+ - "\7$\2\2\u00c7\u00cc\5\30\r\2\u00c8\u00c9\7\f\2\2\u00c9\u00cb\5\30\r\2"+ - "\u00ca\u00c8\3\2\2\2\u00cb\u00ce\3\2\2\2\u00cc\u00ca\3\2\2\2\u00cc\u00cd"+ - "\3\2\2\2\u00cd\u00cf\3\2\2\2\u00ce\u00cc\3\2\2\2\u00cf\u00d0\7&\2\2\u00d0"+ - "\33\3\2\2\2\u00d1\u00d2\b\17\1\2\u00d2\u00d3\t\2\2\2\u00d3\u00ec\5\34"+ - "\17\20\u00d4\u00d5\7\t\2\2\u00d5\u00d6\5\22\n\2\u00d6\u00d7\7\n\2\2\u00d7"+ - "\u00d8\5\34\17\17\u00d8\u00ec\3\2\2\2\u00d9\u00da\5\36\20\2\u00da\u00db"+ - "\t\3\2\2\u00db\u00dc\5\34\17\3\u00dc\u00ec\3\2\2\2\u00dd\u00de\7\t\2\2"+ - "\u00de\u00df\5\34\17\2\u00df\u00e0\7\n\2\2\u00e0\u00ec\3\2\2\2\u00e1\u00ec"+ - "\t\4\2\2\u00e2\u00ec\7F\2\2\u00e3\u00ec\7G\2\2\u00e4\u00ec\7H\2\2\u00e5"+ - "\u00e6\5\36\20\2\u00e6\u00e7\t\5\2\2\u00e7\u00ec\3\2\2\2\u00e8\u00e9\t"+ - "\5\2\2\u00e9\u00ec\5\36\20\2\u00ea\u00ec\5\36\20\2\u00eb\u00d1\3\2\2\2"+ - "\u00eb\u00d4\3\2\2\2\u00eb\u00d9\3\2\2\2\u00eb\u00dd\3\2\2\2\u00eb\u00e1"+ - "\3\2\2\2\u00eb\u00e2\3\2\2\2\u00eb\u00e3\3\2\2\2\u00eb\u00e4\3\2\2\2\u00eb"+ - "\u00e5\3\2\2\2\u00eb\u00e8\3\2\2\2\u00eb\u00ea\3\2\2\2\u00ec\u0113\3\2"+ - "\2\2\u00ed\u00ee\f\16\2\2\u00ee\u00ef\t\6\2\2\u00ef\u0112\5\34\17\17\u00f0"+ - "\u00f1\f\r\2\2\u00f1\u00f2\t\7\2\2\u00f2\u0112\5\34\17\16\u00f3\u00f4"+ - "\f\f\2\2\u00f4\u00f5\t\b\2\2\u00f5\u0112\5\34\17\r\u00f6\u00f7\f\13\2"+ - "\2\u00f7\u00f8\t\t\2\2\u00f8\u0112\5\34\17\f\u00f9\u00fa\f\n\2\2\u00fa"+ - "\u00fb\t\n\2\2\u00fb\u0112\5\34\17\13\u00fc\u00fd\f\t\2\2\u00fd\u00fe"+ - "\7,\2\2\u00fe\u0112\5\34\17\n\u00ff\u0100\f\b\2\2\u0100\u0101\7-\2\2\u0101"+ - "\u0112\5\34\17\t\u0102\u0103\f\7\2\2\u0103\u0104\7.\2\2\u0104\u0112\5"+ - "\34\17\b\u0105\u0106\f\6\2\2\u0106\u0107\7/\2\2\u0107\u0112\5\34\17\7"+ - "\u0108\u0109\f\5\2\2\u0109\u010a\7\60\2\2\u010a\u0112\5\34\17\6\u010b"+ - "\u010c\f\4\2\2\u010c\u010d\7\61\2\2\u010d\u010e\5\34\17\2\u010e\u010f"+ - "\7\62\2\2\u010f\u0110\5\34\17\4\u0110\u0112\3\2\2\2\u0111\u00ed\3\2\2"+ - "\2\u0111\u00f0\3\2\2\2\u0111\u00f3\3\2\2\2\u0111\u00f6\3\2\2\2\u0111\u00f9"+ - "\3\2\2\2\u0111\u00fc\3\2\2\2\u0111\u00ff\3\2\2\2\u0111\u0102\3\2\2\2\u0111"+ - "\u0105\3\2\2\2\u0111\u0108\3\2\2\2\u0111\u010b\3\2\2\2\u0112\u0115\3\2"+ - "\2\2\u0113\u0111\3\2\2\2\u0113\u0114\3\2\2\2\u0114\35\3\2\2\2\u0115\u0113"+ - "\3\2\2\2\u0116\u011c\5 \21\2\u0117\u011c\5\"\22\2\u0118\u011c\5*\26\2"+ - "\u0119\u011c\5.\30\2\u011a\u011c\5\60\31\2\u011b\u0116\3\2\2\2\u011b\u0117"+ - "\3\2\2\2\u011b\u0118\3\2\2\2\u011b\u0119\3\2\2\2\u011b\u011a\3\2\2\2\u011c"+ - "\37\3\2\2\2\u011d\u0123\7\t\2\2\u011e\u0124\5 \21\2\u011f\u0124\5\"\22"+ - "\2\u0120\u0124\5*\26\2\u0121\u0124\5.\30\2\u0122\u0124\5\60\31\2\u0123"+ - "\u011e\3\2\2\2\u0123\u011f\3\2\2\2\u0123\u0120\3\2\2\2\u0123\u0121\3\2"+ - "\2\2\u0123\u0122\3\2\2\2\u0124\u0125\3\2\2\2\u0125\u0128\7\n\2\2\u0126"+ - "\u0129\5&\24\2\u0127\u0129\5$\23\2\u0128\u0126\3\2\2\2\u0128\u0127\3\2"+ - "\2\2\u0128\u0129\3\2\2\2\u0129!\3\2\2\2\u012a\u012b\7\t\2\2\u012b\u012c"+ - "\5\22\n\2\u012c\u0132\7\n\2\2\u012d\u0133\5 \21\2\u012e\u0133\5\"\22\2"+ - "\u012f\u0133\5*\26\2\u0130\u0133\5.\30\2\u0131\u0133\5\60\31\2\u0132\u012d"+ - "\3\2\2\2\u0132\u012e\3\2\2\2\u0132\u012f\3\2\2\2\u0132\u0130\3\2\2\2\u0132"+ - "\u0131\3\2\2\2\u0133#\3\2\2\2\u0134\u0135\7\7\2\2\u0135\u0136\5\34\17"+ - "\2\u0136\u0139\7\b\2\2\u0137\u013a\5&\24\2\u0138\u013a\5$\23\2\u0139\u0137"+ - "\3\2\2\2\u0139\u0138\3\2\2\2\u0139\u013a\3\2\2\2\u013a%\3\2\2\2\u013b"+ - "\u013e\7\13\2\2\u013c\u013f\5(\25\2\u013d\u013f\5,\27\2\u013e\u013c\3"+ - "\2\2\2\u013e\u013d\3\2\2\2\u013f\'\3\2\2\2\u0140\u0141\7K\2\2\u0141\u0144"+ - "\5\62\32\2\u0142\u0145\5&\24\2\u0143\u0145\5$\23\2\u0144\u0142\3\2\2\2"+ - "\u0144\u0143\3\2\2\2\u0144\u0145\3\2\2\2\u0145)\3\2\2\2\u0146\u0149\5"+ - "\30\r\2\u0147\u014a\5&\24\2\u0148\u014a\5$\23\2\u0149\u0147\3\2\2\2\u0149"+ - "\u0148\3\2\2\2\u0149\u014a\3\2\2\2\u014a+\3\2\2\2\u014b\u014e\t\13\2\2"+ - "\u014c\u014f\5&\24\2\u014d\u014f\5$\23\2\u014e\u014c\3\2\2\2\u014e\u014d"+ - "\3\2\2\2\u014e\u014f\3\2\2\2\u014f-\3\2\2\2\u0150\u0151\7\26\2\2\u0151"+ - "\u0161\5\30\r\2\u0152\u0154\5\62\32\2\u0153\u0155\5&\24\2\u0154\u0153"+ - "\3\2\2\2\u0154\u0155\3\2\2\2\u0155\u0162\3\2\2\2\u0156\u0157\7\7\2\2\u0157"+ - "\u0158\5\34\17\2\u0158\u0159\7\b\2\2\u0159\u015b\3\2\2\2\u015a\u0156\3"+ - "\2\2\2\u015b\u015c\3\2\2\2\u015c\u015a\3\2\2\2\u015c\u015d\3\2\2\2\u015d"+ - "\u015f\3\2\2\2\u015e\u0160\5&\24\2\u015f\u015e\3\2\2\2\u015f\u0160\3\2"+ - "\2\2\u0160\u0162\3\2\2\2\u0161\u0152\3\2\2\2\u0161\u015a\3\2\2\2\u0162"+ - "/\3\2\2\2\u0163\u0166\7E\2\2\u0164\u0167\5&\24\2\u0165\u0167\5$\23\2\u0166"+ - "\u0164\3\2\2\2\u0166\u0165\3\2\2\2\u0166\u0167\3\2\2\2\u0167\61\3\2\2"+ - "\2\u0168\u0171\7\t\2\2\u0169\u016e\5\34\17\2\u016a\u016b\7\f\2\2\u016b"+ - "\u016d\5\34\17\2\u016c\u016a\3\2\2\2\u016d\u0170\3\2\2\2\u016e\u016c\3"+ - "\2\2\2\u016e\u016f\3\2\2\2\u016f\u0172\3\2\2\2\u0170\u016e\3\2\2\2\u0171"+ - "\u0169\3\2\2\2\u0171\u0172\3\2\2\2\u0172\u0173\3\2\2\2\u0173\u0174\7\n"+ - "\2\2\u0174\63\3\2\2\2/\67BJSX\\`eimqv}\u0082\u0086\u0088\u008e\u0093\u0097"+ - "\u009e\u00a8\u00b0\u00b6\u00c0\u00c4\u00cc\u00eb\u0111\u0113\u011b\u0123"+ - "\u0128\u0132\u0139\u013e\u0144\u0149\u014e\u0154\u015c\u015f\u0161\u0166"+ - "\u016e\u0171"; + "\3\17\3\17\7\17\u0109\n\17\f\17\16\17\u010c\13\17\3\20\3\20\3\20\3\20"+ + "\3\20\5\20\u0113\n\20\3\21\3\21\3\21\3\21\3\21\3\21\5\21\u011b\n\21\3"+ + "\21\3\21\3\21\5\21\u0120\n\21\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22"+ + "\5\22\u012a\n\22\3\23\3\23\3\23\3\23\3\23\5\23\u0131\n\23\3\24\3\24\3"+ + "\24\5\24\u0136\n\24\3\25\3\25\3\25\3\25\5\25\u013c\n\25\3\26\3\26\3\26"+ + "\5\26\u0141\n\26\3\27\3\27\3\27\5\27\u0146\n\27\3\30\3\30\3\30\3\30\5"+ + "\30\u014c\n\30\3\30\3\30\3\30\3\30\6\30\u0152\n\30\r\30\16\30\u0153\3"+ + "\30\5\30\u0157\n\30\5\30\u0159\n\30\3\31\3\31\3\31\5\31\u015e\n\31\3\32"+ + "\3\32\3\32\3\32\7\32\u0164\n\32\f\32\16\32\u0167\13\32\5\32\u0169\n\32"+ + "\3\32\3\32\3\32\2\3\34\33\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$&"+ + "(*,.\60\62\2\r\3\3\r\r\4\2\32\33\37 \3\2\65@\3\2AD\3\2\63\64\3\2\34\36"+ + "\3\2\37 \3\2!#\3\2$\'\3\2(+\3\2JK\u01a3\2\65\3\2\2\2\4\177\3\2\2\2\6\u008a"+ + "\3\2\2\2\b\u008e\3\2\2\2\n\u0090\3\2\2\2\f\u0095\3\2\2\2\16\u0097\3\2"+ + "\2\2\20\u0099\3\2\2\2\22\u00a2\3\2\2\2\24\u00aa\3\2\2\2\26\u00af\3\2\2"+ + "\2\30\u00b9\3\2\2\2\32\u00bd\3\2\2\2\34\u00e2\3\2\2\2\36\u0112\3\2\2\2"+ + " \u0114\3\2\2\2\"\u0121\3\2\2\2$\u012b\3\2\2\2&\u0132\3\2\2\2(\u0137\3"+ + "\2\2\2*\u013d\3\2\2\2,\u0142\3\2\2\2.\u0147\3\2\2\2\60\u015a\3\2\2\2\62"+ + "\u015f\3\2\2\2\64\66\5\4\3\2\65\64\3\2\2\2\66\67\3\2\2\2\67\65\3\2\2\2"+ + "\678\3\2\2\289\3\2\2\29:\7\2\2\3:\3\3\2\2\2;<\7\16\2\2<=\7\t\2\2=>\5\34"+ + "\17\2>?\7\n\2\2?B\5\6\4\2@A\7\17\2\2AC\5\6\4\2B@\3\2\2\2BC\3\2\2\2C\u0080"+ + "\3\2\2\2DE\7\20\2\2EF\7\t\2\2FG\5\34\17\2GJ\7\n\2\2HK\5\6\4\2IK\5\b\5"+ + "\2JH\3\2\2\2JI\3\2\2\2K\u0080\3\2\2\2LM\7\21\2\2MN\5\6\4\2NO\7\20\2\2"+ + "OP\7\t\2\2PQ\5\34\17\2QR\7\n\2\2RS\t\2\2\2S\u0080\3\2\2\2TU\7\22\2\2U"+ + "W\7\t\2\2VX\5\f\7\2WV\3\2\2\2WX\3\2\2\2XY\3\2\2\2Y[\7\r\2\2Z\\\5\34\17"+ + "\2[Z\3\2\2\2[\\\3\2\2\2\\]\3\2\2\2]_\7\r\2\2^`\5\16\b\2_^\3\2\2\2_`\3"+ + "\2\2\2`a\3\2\2\2ad\7\n\2\2be\5\6\4\2ce\5\b\5\2db\3\2\2\2dc\3\2\2\2e\u0080"+ + "\3\2\2\2fg\5\20\t\2gh\t\2\2\2h\u0080\3\2\2\2ij\7\23\2\2j\u0080\t\2\2\2"+ + "kl\7\24\2\2l\u0080\t\2\2\2mn\7\25\2\2no\5\34\17\2op\t\2\2\2p\u0080\3\2"+ + "\2\2qr\7\27\2\2rt\5\6\4\2su\5\26\f\2ts\3\2\2\2uv\3\2\2\2vt\3\2\2\2vw\3"+ + "\2\2\2w\u0080\3\2\2\2xy\7\31\2\2yz\5\34\17\2z{\t\2\2\2{\u0080\3\2\2\2"+ + "|}\5\34\17\2}~\t\2\2\2~\u0080\3\2\2\2\177;\3\2\2\2\177D\3\2\2\2\177L\3"+ + "\2\2\2\177T\3\2\2\2\177f\3\2\2\2\177i\3\2\2\2\177k\3\2\2\2\177m\3\2\2"+ + "\2\177q\3\2\2\2\177x\3\2\2\2\177|\3\2\2\2\u0080\5\3\2\2\2\u0081\u0083"+ + "\7\5\2\2\u0082\u0084\5\4\3\2\u0083\u0082\3\2\2\2\u0084\u0085\3\2\2\2\u0085"+ + "\u0083\3\2\2\2\u0085\u0086\3\2\2\2\u0086\u0087\3\2\2\2\u0087\u0088\7\6"+ + "\2\2\u0088\u008b\3\2\2\2\u0089\u008b\5\4\3\2\u008a\u0081\3\2\2\2\u008a"+ + "\u0089\3\2\2\2\u008b\7\3\2\2\2\u008c\u008f\5\n\6\2\u008d\u008f\7\r\2\2"+ + "\u008e\u008c\3\2\2\2\u008e\u008d\3\2\2\2\u008f\t\3\2\2\2\u0090\u0091\7"+ + "\5\2\2\u0091\u0092\7\6\2\2\u0092\13\3\2\2\2\u0093\u0096\5\20\t\2\u0094"+ + "\u0096\5\34\17\2\u0095\u0093\3\2\2\2\u0095\u0094\3\2\2\2\u0096\r\3\2\2"+ + "\2\u0097\u0098\5\34\17\2\u0098\17\3\2\2\2\u0099\u009a\5\22\n\2\u009a\u009f"+ + "\5\24\13\2\u009b\u009c\7\f\2\2\u009c\u009e\5\24\13\2\u009d\u009b\3\2\2"+ + "\2\u009e\u00a1\3\2\2\2\u009f\u009d\3\2\2\2\u009f\u00a0\3\2\2\2\u00a0\21"+ + "\3\2\2\2\u00a1\u009f\3\2\2\2\u00a2\u00a7\5\30\r\2\u00a3\u00a4\7\7\2\2"+ + "\u00a4\u00a6\7\b\2\2\u00a5\u00a3\3\2\2\2\u00a6\u00a9\3\2\2\2\u00a7\u00a5"+ + "\3\2\2\2\u00a7\u00a8\3\2\2\2\u00a8\23\3\2\2\2\u00a9\u00a7\3\2\2\2\u00aa"+ + "\u00ad\5\30\r\2\u00ab\u00ac\7\65\2\2\u00ac\u00ae\5\34\17\2\u00ad\u00ab"+ + "\3\2\2\2\u00ad\u00ae\3\2\2\2\u00ae\25\3\2\2\2\u00af\u00b0\7\30\2\2\u00b0"+ + "\u00b1\7\t\2\2\u00b1\u00b2\5\30\r\2\u00b2\u00b3\5\30\r\2\u00b3\u00b4\3"+ + "\2\2\2\u00b4\u00b7\7\n\2\2\u00b5\u00b8\5\6\4\2\u00b6\u00b8\5\n\6\2\u00b7"+ + "\u00b5\3\2\2\2\u00b7\u00b6\3\2\2\2\u00b8\27\3\2\2\2\u00b9\u00bb\7I\2\2"+ + "\u00ba\u00bc\5\32\16\2\u00bb\u00ba\3\2\2\2\u00bb\u00bc\3\2\2\2\u00bc\31"+ + "\3\2\2\2\u00bd\u00be\7$\2\2\u00be\u00c3\5\30\r\2\u00bf\u00c0\7\f\2\2\u00c0"+ + "\u00c2\5\30\r\2\u00c1\u00bf\3\2\2\2\u00c2\u00c5\3\2\2\2\u00c3\u00c1\3"+ + "\2\2\2\u00c3\u00c4\3\2\2\2\u00c4\u00c6\3\2\2\2\u00c5\u00c3\3\2\2\2\u00c6"+ + "\u00c7\7&\2\2\u00c7\33\3\2\2\2\u00c8\u00c9\b\17\1\2\u00c9\u00ca\t\3\2"+ + "\2\u00ca\u00e3\5\34\17\20\u00cb\u00cc\7\t\2\2\u00cc\u00cd\5\22\n\2\u00cd"+ + "\u00ce\7\n\2\2\u00ce\u00cf\5\34\17\17\u00cf\u00e3\3\2\2\2\u00d0\u00d1"+ + "\5\36\20\2\u00d1\u00d2\t\4\2\2\u00d2\u00d3\5\34\17\3\u00d3\u00e3\3\2\2"+ + "\2\u00d4\u00d5\7\t\2\2\u00d5\u00d6\5\34\17\2\u00d6\u00d7\7\n\2\2\u00d7"+ + "\u00e3\3\2\2\2\u00d8\u00e3\t\5\2\2\u00d9\u00e3\7F\2\2\u00da\u00e3\7G\2"+ + "\2\u00db\u00e3\7H\2\2\u00dc\u00dd\5\36\20\2\u00dd\u00de\t\6\2\2\u00de"+ + "\u00e3\3\2\2\2\u00df\u00e0\t\6\2\2\u00e0\u00e3\5\36\20\2\u00e1\u00e3\5"+ + "\36\20\2\u00e2\u00c8\3\2\2\2\u00e2\u00cb\3\2\2\2\u00e2\u00d0\3\2\2\2\u00e2"+ + "\u00d4\3\2\2\2\u00e2\u00d8\3\2\2\2\u00e2\u00d9\3\2\2\2\u00e2\u00da\3\2"+ + "\2\2\u00e2\u00db\3\2\2\2\u00e2\u00dc\3\2\2\2\u00e2\u00df\3\2\2\2\u00e2"+ + "\u00e1\3\2\2\2\u00e3\u010a\3\2\2\2\u00e4\u00e5\f\16\2\2\u00e5\u00e6\t"+ + "\7\2\2\u00e6\u0109\5\34\17\17\u00e7\u00e8\f\r\2\2\u00e8\u00e9\t\b\2\2"+ + "\u00e9\u0109\5\34\17\16\u00ea\u00eb\f\f\2\2\u00eb\u00ec\t\t\2\2\u00ec"+ + "\u0109\5\34\17\r\u00ed\u00ee\f\13\2\2\u00ee\u00ef\t\n\2\2\u00ef\u0109"+ + "\5\34\17\f\u00f0\u00f1\f\n\2\2\u00f1\u00f2\t\13\2\2\u00f2\u0109\5\34\17"+ + "\13\u00f3\u00f4\f\t\2\2\u00f4\u00f5\7,\2\2\u00f5\u0109\5\34\17\n\u00f6"+ + "\u00f7\f\b\2\2\u00f7\u00f8\7-\2\2\u00f8\u0109\5\34\17\t\u00f9\u00fa\f"+ + "\7\2\2\u00fa\u00fb\7.\2\2\u00fb\u0109\5\34\17\b\u00fc\u00fd\f\6\2\2\u00fd"+ + "\u00fe\7/\2\2\u00fe\u0109\5\34\17\7\u00ff\u0100\f\5\2\2\u0100\u0101\7"+ + "\60\2\2\u0101\u0109\5\34\17\6\u0102\u0103\f\4\2\2\u0103\u0104\7\61\2\2"+ + "\u0104\u0105\5\34\17\2\u0105\u0106\7\62\2\2\u0106\u0107\5\34\17\4\u0107"+ + "\u0109\3\2\2\2\u0108\u00e4\3\2\2\2\u0108\u00e7\3\2\2\2\u0108\u00ea\3\2"+ + "\2\2\u0108\u00ed\3\2\2\2\u0108\u00f0\3\2\2\2\u0108\u00f3\3\2\2\2\u0108"+ + "\u00f6\3\2\2\2\u0108\u00f9\3\2\2\2\u0108\u00fc\3\2\2\2\u0108\u00ff\3\2"+ + "\2\2\u0108\u0102\3\2\2\2\u0109\u010c\3\2\2\2\u010a\u0108\3\2\2\2\u010a"+ + "\u010b\3\2\2\2\u010b\35\3\2\2\2\u010c\u010a\3\2\2\2\u010d\u0113\5 \21"+ + "\2\u010e\u0113\5\"\22\2\u010f\u0113\5*\26\2\u0110\u0113\5.\30\2\u0111"+ + "\u0113\5\60\31\2\u0112\u010d\3\2\2\2\u0112\u010e\3\2\2\2\u0112\u010f\3"+ + "\2\2\2\u0112\u0110\3\2\2\2\u0112\u0111\3\2\2\2\u0113\37\3\2\2\2\u0114"+ + "\u011a\7\t\2\2\u0115\u011b\5 \21\2\u0116\u011b\5\"\22\2\u0117\u011b\5"+ + "*\26\2\u0118\u011b\5.\30\2\u0119\u011b\5\60\31\2\u011a\u0115\3\2\2\2\u011a"+ + "\u0116\3\2\2\2\u011a\u0117\3\2\2\2\u011a\u0118\3\2\2\2\u011a\u0119\3\2"+ + "\2\2\u011b\u011c\3\2\2\2\u011c\u011f\7\n\2\2\u011d\u0120\5&\24\2\u011e"+ + "\u0120\5$\23\2\u011f\u011d\3\2\2\2\u011f\u011e\3\2\2\2\u011f\u0120\3\2"+ + "\2\2\u0120!\3\2\2\2\u0121\u0122\7\t\2\2\u0122\u0123\5\22\n\2\u0123\u0129"+ + "\7\n\2\2\u0124\u012a\5 \21\2\u0125\u012a\5\"\22\2\u0126\u012a\5*\26\2"+ + "\u0127\u012a\5.\30\2\u0128\u012a\5\60\31\2\u0129\u0124\3\2\2\2\u0129\u0125"+ + "\3\2\2\2\u0129\u0126\3\2\2\2\u0129\u0127\3\2\2\2\u0129\u0128\3\2\2\2\u012a"+ + "#\3\2\2\2\u012b\u012c\7\7\2\2\u012c\u012d\5\34\17\2\u012d\u0130\7\b\2"+ + "\2\u012e\u0131\5&\24\2\u012f\u0131\5$\23\2\u0130\u012e\3\2\2\2\u0130\u012f"+ + "\3\2\2\2\u0130\u0131\3\2\2\2\u0131%\3\2\2\2\u0132\u0135\7\13\2\2\u0133"+ + "\u0136\5(\25\2\u0134\u0136\5,\27\2\u0135\u0133\3\2\2\2\u0135\u0134\3\2"+ + "\2\2\u0136\'\3\2\2\2\u0137\u0138\7K\2\2\u0138\u013b\5\62\32\2\u0139\u013c"+ + "\5&\24\2\u013a\u013c\5$\23\2\u013b\u0139\3\2\2\2\u013b\u013a\3\2\2\2\u013b"+ + "\u013c\3\2\2\2\u013c)\3\2\2\2\u013d\u0140\5\30\r\2\u013e\u0141\5&\24\2"+ + "\u013f\u0141\5$\23\2\u0140\u013e\3\2\2\2\u0140\u013f\3\2\2\2\u0140\u0141"+ + "\3\2\2\2\u0141+\3\2\2\2\u0142\u0145\t\f\2\2\u0143\u0146\5&\24\2\u0144"+ + "\u0146\5$\23\2\u0145\u0143\3\2\2\2\u0145\u0144\3\2\2\2\u0145\u0146\3\2"+ + "\2\2\u0146-\3\2\2\2\u0147\u0148\7\26\2\2\u0148\u0158\5\30\r\2\u0149\u014b"+ + "\5\62\32\2\u014a\u014c\5&\24\2\u014b\u014a\3\2\2\2\u014b\u014c\3\2\2\2"+ + "\u014c\u0159\3\2\2\2\u014d\u014e\7\7\2\2\u014e\u014f\5\34\17\2\u014f\u0150"+ + "\7\b\2\2\u0150\u0152\3\2\2\2\u0151\u014d\3\2\2\2\u0152\u0153\3\2\2\2\u0153"+ + "\u0151\3\2\2\2\u0153\u0154\3\2\2\2\u0154\u0156\3\2\2\2\u0155\u0157\5&"+ + "\24\2\u0156\u0155\3\2\2\2\u0156\u0157\3\2\2\2\u0157\u0159\3\2\2\2\u0158"+ + "\u0149\3\2\2\2\u0158\u0151\3\2\2\2\u0159/\3\2\2\2\u015a\u015d\7E\2\2\u015b"+ + "\u015e\5&\24\2\u015c\u015e\5$\23\2\u015d\u015b\3\2\2\2\u015d\u015c\3\2"+ + "\2\2\u015d\u015e\3\2\2\2\u015e\61\3\2\2\2\u015f\u0168\7\t\2\2\u0160\u0165"+ + "\5\34\17\2\u0161\u0162\7\f\2\2\u0162\u0164\5\34\17\2\u0163\u0161\3\2\2"+ + "\2\u0164\u0167\3\2\2\2\u0165\u0163\3\2\2\2\u0165\u0166\3\2\2\2\u0166\u0169"+ + "\3\2\2\2\u0167\u0165\3\2\2\2\u0168\u0160\3\2\2\2\u0168\u0169\3\2\2\2\u0169"+ + "\u016a\3\2\2\2\u016a\u016b\7\n\2\2\u016b\63\3\2\2\2(\67BJW[_dv\177\u0085"+ + "\u008a\u008e\u0095\u009f\u00a7\u00ad\u00b7\u00bb\u00c3\u00e2\u0108\u010a"+ + "\u0112\u011a\u011f\u0129\u0130\u0135\u013b\u0140\u0145\u014b\u0153\u0156"+ + "\u0158\u015d\u0165\u0168"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LCall.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LCall.java index 2dd03039ba7..bd76aa293cc 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LCall.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LCall.java @@ -55,15 +55,11 @@ public final class LCall extends ALink { throw new IllegalArgumentException(error("Cannot assign a value to a call [" + name + "].")); } + Definition.MethodKey methodKey = new Definition.MethodKey(name, arguments.size()); final Struct struct = before.struct; - method = statik ? struct.functions.get(name) : struct.methods.get(name); + method = statik ? struct.staticMethods.get(methodKey) : struct.methods.get(methodKey); if (method != null) { - if (method.arguments.size() != arguments.size()) { - throw new IllegalArgumentException(error("When calling [" + name + "] on type [" + struct.name + "]" + - " expected [" + method.arguments.size() + "] arguments, but found [" + arguments.size() + "].")); - } - for (int argument = 0; argument < arguments.size(); ++argument) { final AExpression expression = arguments.get(argument); @@ -83,7 +79,8 @@ public final class LCall extends ALink { return link.analyze(settings, definition, variables); } - throw new IllegalArgumentException(error("Unknown call [" + name + "] on type [" + struct.name + "].")); + throw new IllegalArgumentException(error("Unknown call [" + name + "] with [" + arguments.size() + + "] arguments on type [" + struct.name + "].")); } @Override diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LField.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LField.java index bd0079d1843..06f820eba26 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LField.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LField.java @@ -60,7 +60,7 @@ public final class LField extends ALink { } final Struct struct = before.struct; - field = statik ? struct.statics.get(value) : struct.members.get(value); + field = statik ? struct.staticMembers.get(value) : struct.members.get(value); if (field != null) { if (store && java.lang.reflect.Modifier.isFinal(field.reflect.getModifiers())) { @@ -72,9 +72,12 @@ public final class LField extends ALink { return this; } else { + // TODO: improve this: the isXXX case seems missing??? final boolean shortcut = - struct.methods.containsKey("get" + Character.toUpperCase(value.charAt(0)) + value.substring(1)) || - struct.methods.containsKey("set" + Character.toUpperCase(value.charAt(0)) + value.substring(1)); + struct.methods.containsKey(new Definition.MethodKey("get" + + Character.toUpperCase(value.charAt(0)) + value.substring(1), 0)) || + struct.methods.containsKey(new Definition.MethodKey("set" + + Character.toUpperCase(value.charAt(0)) + value.substring(1), 1)); if (shortcut) { return new LShortcut(line, location, value).copy(this).analyze(settings, definition, variables); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LListShortcut.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LListShortcut.java index 00e44967279..8526ef1297e 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LListShortcut.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LListShortcut.java @@ -43,8 +43,8 @@ final class LListShortcut extends ALink { @Override ALink analyze(final CompilerSettings settings, final Definition definition, final Variables variables) { - getter = before.struct.methods.get("get"); - setter = before.struct.methods.get("set"); + getter = before.struct.methods.get(new Definition.MethodKey("get", 1)); + setter = before.struct.methods.get(new Definition.MethodKey("set", 2)); if (getter != null && (getter.rtn.sort == Sort.VOID || getter.arguments.size() != 1 || getter.arguments.get(0).sort != Sort.INT)) { diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LMapShortcut.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LMapShortcut.java index a3d9d119eda..4efbd4bdf0f 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LMapShortcut.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LMapShortcut.java @@ -43,8 +43,8 @@ final class LMapShortcut extends ALink { @Override ALink analyze(final CompilerSettings settings, final Definition definition, final Variables variables) { - getter = before.struct.methods.get("get"); - setter = before.struct.methods.get("put"); + getter = before.struct.methods.get(new Definition.MethodKey("get", 1)); + setter = before.struct.methods.get(new Definition.MethodKey("put", 2)); if (getter != null && (getter.rtn.sort == Sort.VOID || getter.arguments.size() != 1)) { throw new IllegalArgumentException(error("Illegal map get shortcut for type [" + before.name + "].")); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewObj.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewObj.java index fb91b7a6915..227b63cf31f 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewObj.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewObj.java @@ -63,7 +63,7 @@ public final class LNewObj extends ALink { } final Struct struct = type.struct; - constructor = struct.constructors.get("new"); + constructor = struct.constructors.get(new Definition.MethodKey("new", arguments.size())); if (constructor != null) { final Type[] types = new Type[constructor.arguments.size()]; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LShortcut.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LShortcut.java index 5f18cccd512..c65077e6e28 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LShortcut.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LShortcut.java @@ -47,8 +47,8 @@ final class LShortcut extends ALink { ALink analyze(final CompilerSettings settings, final Definition definition, final Variables variables) { final Struct struct = before.struct; - getter = struct.methods.get("get" + Character.toUpperCase(value.charAt(0)) + value.substring(1)); - setter = struct.methods.get("set" + Character.toUpperCase(value.charAt(0)) + value.substring(1)); + getter = struct.methods.get(new Definition.MethodKey("get" + Character.toUpperCase(value.charAt(0)) + value.substring(1), 0)); + setter = struct.methods.get(new Definition.MethodKey("set" + Character.toUpperCase(value.charAt(0)) + value.substring(1), 1)); if (getter != null && (getter.rtn.sort == Sort.VOID || !getter.arguments.isEmpty())) { throw new IllegalArgumentException(error( diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/DefOperationTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/DefOperationTests.java index cd4bda1c071..22a9d50b991 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/DefOperationTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/DefOperationTests.java @@ -21,894 +21,894 @@ package org.elasticsearch.painless; public class DefOperationTests extends ScriptTestCase { public void testNot() { - assertEquals(~1, exec("def x = (byte)1 return ~x")); - assertEquals(~1, exec("def x = (short)1 return ~x")); - assertEquals(~1, exec("def x = (char)1 return ~x")); - assertEquals(~1, exec("def x = 1 return ~x")); - assertEquals(~1L, exec("def x = 1L return ~x")); + assertEquals(~1, exec("def x = (byte)1; return ~x")); + assertEquals(~1, exec("def x = (short)1; return ~x")); + assertEquals(~1, exec("def x = (char)1; return ~x")); + assertEquals(~1, exec("def x = 1; return ~x")); + assertEquals(~1L, exec("def x = 1L; return ~x")); } public void testNeg() { - assertEquals(-1, exec("def x = (byte)1 return -x")); - assertEquals(-1, exec("def x = (short)1 return -x")); - assertEquals(-1, exec("def x = (char)1 return -x")); - assertEquals(-1, exec("def x = 1 return -x")); - assertEquals(-1L, exec("def x = 1L return -x")); - assertEquals(-1.0F, exec("def x = 1F return -x")); - assertEquals(-1.0, exec("def x = 1.0 return -x")); + assertEquals(-1, exec("def x = (byte)1; return -x")); + assertEquals(-1, exec("def x = (short)1; return -x")); + assertEquals(-1, exec("def x = (char)1; return -x")); + assertEquals(-1, exec("def x = 1; return -x")); + assertEquals(-1L, exec("def x = 1L; return -x")); + assertEquals(-1.0F, exec("def x = 1F; return -x")); + assertEquals(-1.0, exec("def x = 1.0; return -x")); } public void testMul() { - assertEquals(4, exec("def x = (byte)2 def y = (byte)2 return x * y")); - assertEquals(4, exec("def x = (short)2 def y = (byte)2 return x * y")); - assertEquals(4, exec("def x = (char)2 def y = (byte)2 return x * y")); - assertEquals(4, exec("def x = (int)2 def y = (byte)2 return x * y")); - assertEquals(4L, exec("def x = (long)2 def y = (byte)2 return x * y")); - assertEquals(4F, exec("def x = (float)2 def y = (byte)2 return x * y")); - assertEquals(4D, exec("def x = (double)2 def y = (byte)2 return x * y")); + assertEquals(4, exec("def x = (byte)2; def y = (byte)2; return x * y")); + assertEquals(4, exec("def x = (short)2; def y = (byte)2; return x * y")); + assertEquals(4, exec("def x = (char)2; def y = (byte)2; return x * y")); + assertEquals(4, exec("def x = (int)2; def y = (byte)2; return x * y")); + assertEquals(4L, exec("def x = (long)2; def y = (byte)2; return x * y")); + assertEquals(4F, exec("def x = (float)2; def y = (byte)2; return x * y")); + assertEquals(4D, exec("def x = (double)2; def y = (byte)2; return x * y")); - assertEquals(4, exec("def x = (byte)2 def y = (short)2 return x * y")); - assertEquals(4, exec("def x = (short)2 def y = (short)2 return x * y")); - assertEquals(4, exec("def x = (char)2 def y = (short)2 return x * y")); - assertEquals(4, exec("def x = (int)2 def y = (short)2 return x * y")); - assertEquals(4L, exec("def x = (long)2 def y = (short)2 return x * y")); - assertEquals(4F, exec("def x = (float)2 def y = (short)2 return x * y")); - assertEquals(4D, exec("def x = (double)2 def y = (short)2 return x * y")); + assertEquals(4, exec("def x = (byte)2; def y = (short)2; return x * y")); + assertEquals(4, exec("def x = (short)2; def y = (short)2; return x * y")); + assertEquals(4, exec("def x = (char)2; def y = (short)2; return x * y")); + assertEquals(4, exec("def x = (int)2; def y = (short)2; return x * y")); + assertEquals(4L, exec("def x = (long)2; def y = (short)2; return x * y")); + assertEquals(4F, exec("def x = (float)2; def y = (short)2; return x * y")); + assertEquals(4D, exec("def x = (double)2; def y = (short)2; return x * y")); - assertEquals(4, exec("def x = (byte)2 def y = (char)2 return x * y")); - assertEquals(4, exec("def x = (short)2 def y = (char)2 return x * y")); - assertEquals(4, exec("def x = (char)2 def y = (char)2 return x * y")); - assertEquals(4, exec("def x = (int)2 def y = (char)2 return x * y")); - assertEquals(4L, exec("def x = (long)2 def y = (char)2 return x * y")); - assertEquals(4F, exec("def x = (float)2 def y = (char)2 return x * y")); - assertEquals(4D, exec("def x = (double)2 def y = (char)2 return x * y")); + assertEquals(4, exec("def x = (byte)2; def y = (char)2; return x * y")); + assertEquals(4, exec("def x = (short)2; def y = (char)2; return x * y")); + assertEquals(4, exec("def x = (char)2; def y = (char)2; return x * y")); + assertEquals(4, exec("def x = (int)2; def y = (char)2; return x * y")); + assertEquals(4L, exec("def x = (long)2; def y = (char)2; return x * y")); + assertEquals(4F, exec("def x = (float)2; def y = (char)2; return x * y")); + assertEquals(4D, exec("def x = (double)2; def y = (char)2; return x * y")); - assertEquals(4, exec("def x = (byte)2 def y = (int)2 return x * y")); - assertEquals(4, exec("def x = (short)2 def y = (int)2 return x * y")); - assertEquals(4, exec("def x = (char)2 def y = (int)2 return x * y")); - assertEquals(4, exec("def x = (int)2 def y = (int)2 return x * y")); - assertEquals(4L, exec("def x = (long)2 def y = (int)2 return x * y")); - assertEquals(4F, exec("def x = (float)2 def y = (int)2 return x * y")); - assertEquals(4D, exec("def x = (double)2 def y = (int)2 return x * y")); + assertEquals(4, exec("def x = (byte)2; def y = (int)2; return x * y")); + assertEquals(4, exec("def x = (short)2; def y = (int)2; return x * y")); + assertEquals(4, exec("def x = (char)2; def y = (int)2; return x * y")); + assertEquals(4, exec("def x = (int)2; def y = (int)2; return x * y")); + assertEquals(4L, exec("def x = (long)2; def y = (int)2; return x * y")); + assertEquals(4F, exec("def x = (float)2; def y = (int)2; return x * y")); + assertEquals(4D, exec("def x = (double)2; def y = (int)2; return x * y")); - assertEquals(4L, exec("def x = (byte)2 def y = (long)2 return x * y")); - assertEquals(4L, exec("def x = (short)2 def y = (long)2 return x * y")); - assertEquals(4L, exec("def x = (char)2 def y = (long)2 return x * y")); - assertEquals(4L, exec("def x = (int)2 def y = (long)2 return x * y")); - assertEquals(4L, exec("def x = (long)2 def y = (long)2 return x * y")); - assertEquals(4F, exec("def x = (float)2 def y = (long)2 return x * y")); - assertEquals(4D, exec("def x = (double)2 def y = (long)2 return x * y")); + assertEquals(4L, exec("def x = (byte)2; def y = (long)2; return x * y")); + assertEquals(4L, exec("def x = (short)2; def y = (long)2; return x * y")); + assertEquals(4L, exec("def x = (char)2; def y = (long)2; return x * y")); + assertEquals(4L, exec("def x = (int)2; def y = (long)2; return x * y")); + assertEquals(4L, exec("def x = (long)2; def y = (long)2; return x * y")); + assertEquals(4F, exec("def x = (float)2; def y = (long)2; return x * y")); + assertEquals(4D, exec("def x = (double)2; def y = (long)2; return x * y")); - assertEquals(4F, exec("def x = (byte)2 def y = (float)2 return x * y")); - assertEquals(4F, exec("def x = (short)2 def y = (float)2 return x * y")); - assertEquals(4F, exec("def x = (char)2 def y = (float)2 return x * y")); - assertEquals(4F, exec("def x = (int)2 def y = (float)2 return x * y")); - assertEquals(4F, exec("def x = (long)2 def y = (float)2 return x * y")); - assertEquals(4F, exec("def x = (float)2 def y = (float)2 return x * y")); - assertEquals(4D, exec("def x = (double)2 def y = (float)2 return x * y")); + assertEquals(4F, exec("def x = (byte)2; def y = (float)2; return x * y")); + assertEquals(4F, exec("def x = (short)2; def y = (float)2; return x * y")); + assertEquals(4F, exec("def x = (char)2; def y = (float)2; return x * y")); + assertEquals(4F, exec("def x = (int)2; def y = (float)2; return x * y")); + assertEquals(4F, exec("def x = (long)2; def y = (float)2; return x * y")); + assertEquals(4F, exec("def x = (float)2; def y = (float)2; return x * y")); + assertEquals(4D, exec("def x = (double)2; def y = (float)2; return x * y")); - assertEquals(4D, exec("def x = (byte)2 def y = (double)2 return x * y")); - assertEquals(4D, exec("def x = (short)2 def y = (double)2 return x * y")); - assertEquals(4D, exec("def x = (char)2 def y = (double)2 return x * y")); - assertEquals(4D, exec("def x = (int)2 def y = (double)2 return x * y")); - assertEquals(4D, exec("def x = (long)2 def y = (double)2 return x * y")); - assertEquals(4D, exec("def x = (float)2 def y = (double)2 return x * y")); - assertEquals(4D, exec("def x = (double)2 def y = (double)2 return x * y")); + assertEquals(4D, exec("def x = (byte)2; def y = (double)2; return x * y")); + assertEquals(4D, exec("def x = (short)2; def y = (double)2; return x * y")); + assertEquals(4D, exec("def x = (char)2; def y = (double)2; return x * y")); + assertEquals(4D, exec("def x = (int)2; def y = (double)2; return x * y")); + assertEquals(4D, exec("def x = (long)2; def y = (double)2; return x * y")); + assertEquals(4D, exec("def x = (float)2; def y = (double)2; return x * y")); + assertEquals(4D, exec("def x = (double)2; def y = (double)2; return x * y")); - assertEquals(4, exec("def x = (Byte)2 def y = (byte)2 return x * y")); - assertEquals(4, exec("def x = (Short)2 def y = (short)2 return x * y")); - assertEquals(4, exec("def x = (Character)2 def y = (char)2 return x * y")); - assertEquals(4, exec("def x = (Integer)2 def y = (int)2 return x * y")); - assertEquals(4L, exec("def x = (Long)2 def y = (long)2 return x * y")); - assertEquals(4F, exec("def x = (Float)2 def y = (float)2 return x * y")); - assertEquals(4D, exec("def x = (Double)2 def y = (double)2 return x * y")); + assertEquals(4, exec("def x = (Byte)2; def y = (byte)2; return x * y")); + assertEquals(4, exec("def x = (Short)2; def y = (short)2; return x * y")); + assertEquals(4, exec("def x = (Character)2; def y = (char)2; return x * y")); + assertEquals(4, exec("def x = (Integer)2; def y = (int)2; return x * y")); + assertEquals(4L, exec("def x = (Long)2; def y = (long)2; return x * y")); + assertEquals(4F, exec("def x = (Float)2; def y = (float)2; return x * y")); + assertEquals(4D, exec("def x = (Double)2; def y = (double)2; return x * y")); } public void testDiv() { - assertEquals(1, exec("def x = (byte)2 def y = (byte)2 return x / y")); - assertEquals(1, exec("def x = (short)2 def y = (byte)2 return x / y")); - assertEquals(1, exec("def x = (char)2 def y = (byte)2 return x / y")); - assertEquals(1, exec("def x = (int)2 def y = (byte)2 return x / y")); - assertEquals(1L, exec("def x = (long)2 def y = (byte)2 return x / y")); - assertEquals(1F, exec("def x = (float)2 def y = (byte)2 return x / y")); - assertEquals(1D, exec("def x = (double)2 def y = (byte)2 return x / y")); + assertEquals(1, exec("def x = (byte)2; def y = (byte)2; return x / y")); + assertEquals(1, exec("def x = (short)2; def y = (byte)2; return x / y")); + assertEquals(1, exec("def x = (char)2; def y = (byte)2; return x / y")); + assertEquals(1, exec("def x = (int)2; def y = (byte)2; return x / y")); + assertEquals(1L, exec("def x = (long)2; def y = (byte)2; return x / y")); + assertEquals(1F, exec("def x = (float)2; def y = (byte)2; return x / y")); + assertEquals(1D, exec("def x = (double)2; def y = (byte)2; return x / y")); - assertEquals(1, exec("def x = (byte)2 def y = (short)2 return x / y")); - assertEquals(1, exec("def x = (short)2 def y = (short)2 return x / y")); - assertEquals(1, exec("def x = (char)2 def y = (short)2 return x / y")); - assertEquals(1, exec("def x = (int)2 def y = (short)2 return x / y")); - assertEquals(1L, exec("def x = (long)2 def y = (short)2 return x / y")); - assertEquals(1F, exec("def x = (float)2 def y = (short)2 return x / y")); - assertEquals(1D, exec("def x = (double)2 def y = (short)2 return x / y")); + assertEquals(1, exec("def x = (byte)2; def y = (short)2; return x / y")); + assertEquals(1, exec("def x = (short)2; def y = (short)2; return x / y")); + assertEquals(1, exec("def x = (char)2; def y = (short)2; return x / y")); + assertEquals(1, exec("def x = (int)2; def y = (short)2; return x / y")); + assertEquals(1L, exec("def x = (long)2; def y = (short)2; return x / y")); + assertEquals(1F, exec("def x = (float)2; def y = (short)2; return x / y")); + assertEquals(1D, exec("def x = (double)2; def y = (short)2; return x / y")); - assertEquals(1, exec("def x = (byte)2 def y = (char)2 return x / y")); - assertEquals(1, exec("def x = (short)2 def y = (char)2 return x / y")); - assertEquals(1, exec("def x = (char)2 def y = (char)2 return x / y")); - assertEquals(1, exec("def x = (int)2 def y = (char)2 return x / y")); - assertEquals(1L, exec("def x = (long)2 def y = (char)2 return x / y")); - assertEquals(1F, exec("def x = (float)2 def y = (char)2 return x / y")); - assertEquals(1D, exec("def x = (double)2 def y = (char)2 return x / y")); + assertEquals(1, exec("def x = (byte)2; def y = (char)2; return x / y")); + assertEquals(1, exec("def x = (short)2; def y = (char)2; return x / y")); + assertEquals(1, exec("def x = (char)2; def y = (char)2; return x / y")); + assertEquals(1, exec("def x = (int)2; def y = (char)2; return x / y")); + assertEquals(1L, exec("def x = (long)2; def y = (char)2; return x / y")); + assertEquals(1F, exec("def x = (float)2; def y = (char)2; return x / y")); + assertEquals(1D, exec("def x = (double)2; def y = (char)2; return x / y")); - assertEquals(1, exec("def x = (byte)2 def y = (int)2 return x / y")); - assertEquals(1, exec("def x = (short)2 def y = (int)2 return x / y")); - assertEquals(1, exec("def x = (char)2 def y = (int)2 return x / y")); - assertEquals(1, exec("def x = (int)2 def y = (int)2 return x / y")); - assertEquals(1L, exec("def x = (long)2 def y = (int)2 return x / y")); - assertEquals(1F, exec("def x = (float)2 def y = (int)2 return x / y")); - assertEquals(1D, exec("def x = (double)2 def y = (int)2 return x / y")); + assertEquals(1, exec("def x = (byte)2; def y = (int)2; return x / y")); + assertEquals(1, exec("def x = (short)2; def y = (int)2; return x / y")); + assertEquals(1, exec("def x = (char)2; def y = (int)2; return x / y")); + assertEquals(1, exec("def x = (int)2; def y = (int)2; return x / y")); + assertEquals(1L, exec("def x = (long)2; def y = (int)2; return x / y")); + assertEquals(1F, exec("def x = (float)2; def y = (int)2; return x / y")); + assertEquals(1D, exec("def x = (double)2; def y = (int)2; return x / y")); - assertEquals(1L, exec("def x = (byte)2 def y = (long)2 return x / y")); - assertEquals(1L, exec("def x = (short)2 def y = (long)2 return x / y")); - assertEquals(1L, exec("def x = (char)2 def y = (long)2 return x / y")); - assertEquals(1L, exec("def x = (int)2 def y = (long)2 return x / y")); - assertEquals(1L, exec("def x = (long)2 def y = (long)2 return x / y")); - assertEquals(1F, exec("def x = (float)2 def y = (long)2 return x / y")); - assertEquals(1D, exec("def x = (double)2 def y = (long)2 return x / y")); + assertEquals(1L, exec("def x = (byte)2; def y = (long)2; return x / y")); + assertEquals(1L, exec("def x = (short)2; def y = (long)2; return x / y")); + assertEquals(1L, exec("def x = (char)2; def y = (long)2; return x / y")); + assertEquals(1L, exec("def x = (int)2; def y = (long)2; return x / y")); + assertEquals(1L, exec("def x = (long)2; def y = (long)2; return x / y")); + assertEquals(1F, exec("def x = (float)2; def y = (long)2; return x / y")); + assertEquals(1D, exec("def x = (double)2; def y = (long)2; return x / y")); - assertEquals(1F, exec("def x = (byte)2 def y = (float)2 return x / y")); - assertEquals(1F, exec("def x = (short)2 def y = (float)2 return x / y")); - assertEquals(1F, exec("def x = (char)2 def y = (float)2 return x / y")); - assertEquals(1F, exec("def x = (int)2 def y = (float)2 return x / y")); - assertEquals(1F, exec("def x = (long)2 def y = (float)2 return x / y")); - assertEquals(1F, exec("def x = (float)2 def y = (float)2 return x / y")); - assertEquals(1D, exec("def x = (double)2 def y = (float)2 return x / y")); + assertEquals(1F, exec("def x = (byte)2; def y = (float)2; return x / y")); + assertEquals(1F, exec("def x = (short)2; def y = (float)2; return x / y")); + assertEquals(1F, exec("def x = (char)2; def y = (float)2; return x / y")); + assertEquals(1F, exec("def x = (int)2; def y = (float)2; return x / y")); + assertEquals(1F, exec("def x = (long)2; def y = (float)2; return x / y")); + assertEquals(1F, exec("def x = (float)2; def y = (float)2; return x / y")); + assertEquals(1D, exec("def x = (double)2; def y = (float)2; return x / y")); - assertEquals(1D, exec("def x = (byte)2 def y = (double)2 return x / y")); - assertEquals(1D, exec("def x = (short)2 def y = (double)2 return x / y")); - assertEquals(1D, exec("def x = (char)2 def y = (double)2 return x / y")); - assertEquals(1D, exec("def x = (int)2 def y = (double)2 return x / y")); - assertEquals(1D, exec("def x = (long)2 def y = (double)2 return x / y")); - assertEquals(1D, exec("def x = (float)2 def y = (double)2 return x / y")); - assertEquals(1D, exec("def x = (double)2 def y = (double)2 return x / y")); + assertEquals(1D, exec("def x = (byte)2; def y = (double)2; return x / y")); + assertEquals(1D, exec("def x = (short)2; def y = (double)2; return x / y")); + assertEquals(1D, exec("def x = (char)2; def y = (double)2; return x / y")); + assertEquals(1D, exec("def x = (int)2; def y = (double)2; return x / y")); + assertEquals(1D, exec("def x = (long)2; def y = (double)2; return x / y")); + assertEquals(1D, exec("def x = (float)2; def y = (double)2; return x / y")); + assertEquals(1D, exec("def x = (double)2; def y = (double)2; return x / y")); - assertEquals(1, exec("def x = (Byte)2 def y = (byte)2 return x / y")); - assertEquals(1, exec("def x = (Short)2 def y = (short)2 return x / y")); - assertEquals(1, exec("def x = (Character)2 def y = (char)2 return x / y")); - assertEquals(1, exec("def x = (Integer)2 def y = (int)2 return x / y")); - assertEquals(1L, exec("def x = (Long)2 def y = (long)2 return x / y")); - assertEquals(1F, exec("def x = (Float)2 def y = (float)2 return x / y")); - assertEquals(1D, exec("def x = (Double)2 def y = (double)2 return x / y")); + assertEquals(1, exec("def x = (Byte)2; def y = (byte)2; return x / y")); + assertEquals(1, exec("def x = (Short)2; def y = (short)2; return x / y")); + assertEquals(1, exec("def x = (Character)2; def y = (char)2; return x / y")); + assertEquals(1, exec("def x = (Integer)2; def y = (int)2; return x / y")); + assertEquals(1L, exec("def x = (Long)2; def y = (long)2; return x / y")); + assertEquals(1F, exec("def x = (Float)2; def y = (float)2; return x / y")); + assertEquals(1D, exec("def x = (Double)2; def y = (double)2; return x / y")); } public void testRem() { - assertEquals(0, exec("def x = (byte)2 def y = (byte)2 return x % y")); - assertEquals(0, exec("def x = (short)2 def y = (byte)2 return x % y")); - assertEquals(0, exec("def x = (char)2 def y = (byte)2 return x % y")); - assertEquals(0, exec("def x = (int)2 def y = (byte)2 return x % y")); - assertEquals(0L, exec("def x = (long)2 def y = (byte)2 return x % y")); - assertEquals(0F, exec("def x = (float)2 def y = (byte)2 return x % y")); - assertEquals(0D, exec("def x = (double)2 def y = (byte)2 return x % y")); + assertEquals(0, exec("def x = (byte)2; def y = (byte)2; return x % y")); + assertEquals(0, exec("def x = (short)2; def y = (byte)2; return x % y")); + assertEquals(0, exec("def x = (char)2; def y = (byte)2; return x % y")); + assertEquals(0, exec("def x = (int)2; def y = (byte)2; return x % y")); + assertEquals(0L, exec("def x = (long)2; def y = (byte)2; return x % y")); + assertEquals(0F, exec("def x = (float)2; def y = (byte)2; return x % y")); + assertEquals(0D, exec("def x = (double)2; def y = (byte)2; return x % y")); - assertEquals(0, exec("def x = (byte)2 def y = (short)2 return x % y")); - assertEquals(0, exec("def x = (short)2 def y = (short)2 return x % y")); - assertEquals(0, exec("def x = (char)2 def y = (short)2 return x % y")); - assertEquals(0, exec("def x = (int)2 def y = (short)2 return x % y")); - assertEquals(0L, exec("def x = (long)2 def y = (short)2 return x % y")); - assertEquals(0F, exec("def x = (float)2 def y = (short)2 return x % y")); - assertEquals(0D, exec("def x = (double)2 def y = (short)2 return x % y")); + assertEquals(0, exec("def x = (byte)2; def y = (short)2; return x % y")); + assertEquals(0, exec("def x = (short)2; def y = (short)2; return x % y")); + assertEquals(0, exec("def x = (char)2; def y = (short)2; return x % y")); + assertEquals(0, exec("def x = (int)2; def y = (short)2; return x % y")); + assertEquals(0L, exec("def x = (long)2; def y = (short)2; return x % y")); + assertEquals(0F, exec("def x = (float)2; def y = (short)2; return x % y")); + assertEquals(0D, exec("def x = (double)2; def y = (short)2; return x % y")); - assertEquals(0, exec("def x = (byte)2 def y = (char)2 return x % y")); - assertEquals(0, exec("def x = (short)2 def y = (char)2 return x % y")); - assertEquals(0, exec("def x = (char)2 def y = (char)2 return x % y")); - assertEquals(0, exec("def x = (int)2 def y = (char)2 return x % y")); - assertEquals(0L, exec("def x = (long)2 def y = (char)2 return x % y")); - assertEquals(0F, exec("def x = (float)2 def y = (char)2 return x % y")); - assertEquals(0D, exec("def x = (double)2 def y = (char)2 return x % y")); + assertEquals(0, exec("def x = (byte)2; def y = (char)2; return x % y")); + assertEquals(0, exec("def x = (short)2; def y = (char)2; return x % y")); + assertEquals(0, exec("def x = (char)2; def y = (char)2; return x % y")); + assertEquals(0, exec("def x = (int)2; def y = (char)2; return x % y")); + assertEquals(0L, exec("def x = (long)2; def y = (char)2; return x % y")); + assertEquals(0F, exec("def x = (float)2; def y = (char)2; return x % y")); + assertEquals(0D, exec("def x = (double)2; def y = (char)2; return x % y")); - assertEquals(0, exec("def x = (byte)2 def y = (int)2 return x % y")); - assertEquals(0, exec("def x = (short)2 def y = (int)2 return x % y")); - assertEquals(0, exec("def x = (char)2 def y = (int)2 return x % y")); - assertEquals(0, exec("def x = (int)2 def y = (int)2 return x % y")); - assertEquals(0L, exec("def x = (long)2 def y = (int)2 return x % y")); - assertEquals(0F, exec("def x = (float)2 def y = (int)2 return x % y")); - assertEquals(0D, exec("def x = (double)2 def y = (int)2 return x % y")); + assertEquals(0, exec("def x = (byte)2; def y = (int)2; return x % y")); + assertEquals(0, exec("def x = (short)2; def y = (int)2; return x % y")); + assertEquals(0, exec("def x = (char)2; def y = (int)2; return x % y")); + assertEquals(0, exec("def x = (int)2; def y = (int)2; return x % y")); + assertEquals(0L, exec("def x = (long)2; def y = (int)2; return x % y")); + assertEquals(0F, exec("def x = (float)2; def y = (int)2; return x % y")); + assertEquals(0D, exec("def x = (double)2; def y = (int)2; return x % y")); - assertEquals(0L, exec("def x = (byte)2 def y = (long)2 return x % y")); - assertEquals(0L, exec("def x = (short)2 def y = (long)2 return x % y")); - assertEquals(0L, exec("def x = (char)2 def y = (long)2 return x % y")); - assertEquals(0L, exec("def x = (int)2 def y = (long)2 return x % y")); - assertEquals(0L, exec("def x = (long)2 def y = (long)2 return x % y")); - assertEquals(0F, exec("def x = (float)2 def y = (long)2 return x % y")); - assertEquals(0D, exec("def x = (double)2 def y = (long)2 return x % y")); + assertEquals(0L, exec("def x = (byte)2; def y = (long)2; return x % y")); + assertEquals(0L, exec("def x = (short)2; def y = (long)2; return x % y")); + assertEquals(0L, exec("def x = (char)2; def y = (long)2; return x % y")); + assertEquals(0L, exec("def x = (int)2; def y = (long)2; return x % y")); + assertEquals(0L, exec("def x = (long)2; def y = (long)2; return x % y")); + assertEquals(0F, exec("def x = (float)2; def y = (long)2; return x % y")); + assertEquals(0D, exec("def x = (double)2; def y = (long)2; return x % y")); - assertEquals(0F, exec("def x = (byte)2 def y = (float)2 return x % y")); - assertEquals(0F, exec("def x = (short)2 def y = (float)2 return x % y")); - assertEquals(0F, exec("def x = (char)2 def y = (float)2 return x % y")); - assertEquals(0F, exec("def x = (int)2 def y = (float)2 return x % y")); - assertEquals(0F, exec("def x = (long)2 def y = (float)2 return x % y")); - assertEquals(0F, exec("def x = (float)2 def y = (float)2 return x % y")); - assertEquals(0D, exec("def x = (double)2 def y = (float)2 return x % y")); + assertEquals(0F, exec("def x = (byte)2; def y = (float)2; return x % y")); + assertEquals(0F, exec("def x = (short)2; def y = (float)2; return x % y")); + assertEquals(0F, exec("def x = (char)2; def y = (float)2; return x % y")); + assertEquals(0F, exec("def x = (int)2; def y = (float)2; return x % y")); + assertEquals(0F, exec("def x = (long)2; def y = (float)2; return x % y")); + assertEquals(0F, exec("def x = (float)2; def y = (float)2; return x % y")); + assertEquals(0D, exec("def x = (double)2; def y = (float)2; return x % y")); - assertEquals(0D, exec("def x = (byte)2 def y = (double)2 return x % y")); - assertEquals(0D, exec("def x = (short)2 def y = (double)2 return x % y")); - assertEquals(0D, exec("def x = (char)2 def y = (double)2 return x % y")); - assertEquals(0D, exec("def x = (int)2 def y = (double)2 return x % y")); - assertEquals(0D, exec("def x = (long)2 def y = (double)2 return x % y")); - assertEquals(0D, exec("def x = (float)2 def y = (double)2 return x % y")); - assertEquals(0D, exec("def x = (double)2 def y = (double)2 return x % y")); + assertEquals(0D, exec("def x = (byte)2; def y = (double)2; return x % y")); + assertEquals(0D, exec("def x = (short)2; def y = (double)2; return x % y")); + assertEquals(0D, exec("def x = (char)2; def y = (double)2; return x % y")); + assertEquals(0D, exec("def x = (int)2; def y = (double)2; return x % y")); + assertEquals(0D, exec("def x = (long)2; def y = (double)2; return x % y")); + assertEquals(0D, exec("def x = (float)2; def y = (double)2; return x % y")); + assertEquals(0D, exec("def x = (double)2; def y = (double)2; return x % y")); - assertEquals(0, exec("def x = (Byte)2 def y = (byte)2 return x % y")); - assertEquals(0, exec("def x = (Short)2 def y = (short)2 return x % y")); - assertEquals(0, exec("def x = (Character)2 def y = (char)2 return x % y")); - assertEquals(0, exec("def x = (Integer)2 def y = (int)2 return x % y")); - assertEquals(0L, exec("def x = (Long)2 def y = (long)2 return x % y")); - assertEquals(0F, exec("def x = (Float)2 def y = (float)2 return x % y")); - assertEquals(0D, exec("def x = (Double)2 def y = (double)2 return x % y")); + assertEquals(0, exec("def x = (Byte)2; def y = (byte)2; return x % y")); + assertEquals(0, exec("def x = (Short)2; def y = (short)2; return x % y")); + assertEquals(0, exec("def x = (Character)2; def y = (char)2; return x % y")); + assertEquals(0, exec("def x = (Integer)2; def y = (int)2; return x % y")); + assertEquals(0L, exec("def x = (Long)2; def y = (long)2; return x % y")); + assertEquals(0F, exec("def x = (Float)2; def y = (float)2; return x % y")); + assertEquals(0D, exec("def x = (Double)2; def y = (double)2; return x % y")); } public void testAdd() { - assertEquals(2, exec("def x = (byte)1 def y = (byte)1 return x + y")); - assertEquals(2, exec("def x = (short)1 def y = (byte)1 return x + y")); - assertEquals(2, exec("def x = (char)1 def y = (byte)1 return x + y")); - assertEquals(2, exec("def x = (int)1 def y = (byte)1 return x + y")); - assertEquals(2L, exec("def x = (long)1 def y = (byte)1 return x + y")); - assertEquals(2F, exec("def x = (float)1 def y = (byte)1 return x + y")); - assertEquals(2D, exec("def x = (double)1 def y = (byte)1 return x + y")); + assertEquals(2, exec("def x = (byte)1; def y = (byte)1; return x + y")); + assertEquals(2, exec("def x = (short)1; def y = (byte)1; return x + y")); + assertEquals(2, exec("def x = (char)1; def y = (byte)1; return x + y")); + assertEquals(2, exec("def x = (int)1; def y = (byte)1; return x + y")); + assertEquals(2L, exec("def x = (long)1; def y = (byte)1; return x + y")); + assertEquals(2F, exec("def x = (float)1; def y = (byte)1; return x + y")); + assertEquals(2D, exec("def x = (double)1; def y = (byte)1; return x + y")); - assertEquals(2, exec("def x = (byte)1 def y = (short)1 return x + y")); - assertEquals(2, exec("def x = (short)1 def y = (short)1 return x + y")); - assertEquals(2, exec("def x = (char)1 def y = (short)1 return x + y")); - assertEquals(2, exec("def x = (int)1 def y = (short)1 return x + y")); - assertEquals(2L, exec("def x = (long)1 def y = (short)1 return x + y")); - assertEquals(2F, exec("def x = (float)1 def y = (short)1 return x + y")); - assertEquals(2D, exec("def x = (double)1 def y = (short)1 return x + y")); + assertEquals(2, exec("def x = (byte)1; def y = (short)1; return x + y")); + assertEquals(2, exec("def x = (short)1; def y = (short)1; return x + y")); + assertEquals(2, exec("def x = (char)1; def y = (short)1; return x + y")); + assertEquals(2, exec("def x = (int)1; def y = (short)1; return x + y")); + assertEquals(2L, exec("def x = (long)1; def y = (short)1; return x + y")); + assertEquals(2F, exec("def x = (float)1; def y = (short)1; return x + y")); + assertEquals(2D, exec("def x = (double)1; def y = (short)1; return x + y")); - assertEquals(2, exec("def x = (byte)1 def y = (char)1 return x + y")); - assertEquals(2, exec("def x = (short)1 def y = (char)1 return x + y")); - assertEquals(2, exec("def x = (char)1 def y = (char)1 return x + y")); - assertEquals(2, exec("def x = (int)1 def y = (char)1 return x + y")); - assertEquals(2L, exec("def x = (long)1 def y = (char)1 return x + y")); - assertEquals(2F, exec("def x = (float)1 def y = (char)1 return x + y")); - assertEquals(2D, exec("def x = (double)1 def y = (char)1 return x + y")); + assertEquals(2, exec("def x = (byte)1; def y = (char)1; return x + y")); + assertEquals(2, exec("def x = (short)1; def y = (char)1; return x + y")); + assertEquals(2, exec("def x = (char)1; def y = (char)1; return x + y")); + assertEquals(2, exec("def x = (int)1; def y = (char)1; return x + y")); + assertEquals(2L, exec("def x = (long)1; def y = (char)1; return x + y")); + assertEquals(2F, exec("def x = (float)1; def y = (char)1; return x + y")); + assertEquals(2D, exec("def x = (double)1; def y = (char)1; return x + y")); - assertEquals(2, exec("def x = (byte)1 def y = (int)1 return x + y")); - assertEquals(2, exec("def x = (short)1 def y = (int)1 return x + y")); - assertEquals(2, exec("def x = (char)1 def y = (int)1 return x + y")); - assertEquals(2, exec("def x = (int)1 def y = (int)1 return x + y")); - assertEquals(2L, exec("def x = (long)1 def y = (int)1 return x + y")); - assertEquals(2F, exec("def x = (float)1 def y = (int)1 return x + y")); - assertEquals(2D, exec("def x = (double)1 def y = (int)1 return x + y")); + assertEquals(2, exec("def x = (byte)1; def y = (int)1; return x + y")); + assertEquals(2, exec("def x = (short)1; def y = (int)1; return x + y")); + assertEquals(2, exec("def x = (char)1; def y = (int)1; return x + y")); + assertEquals(2, exec("def x = (int)1; def y = (int)1; return x + y")); + assertEquals(2L, exec("def x = (long)1; def y = (int)1; return x + y")); + assertEquals(2F, exec("def x = (float)1; def y = (int)1; return x + y")); + assertEquals(2D, exec("def x = (double)1; def y = (int)1; return x + y")); - assertEquals(2L, exec("def x = (byte)1 def y = (long)1 return x + y")); - assertEquals(2L, exec("def x = (short)1 def y = (long)1 return x + y")); - assertEquals(2L, exec("def x = (char)1 def y = (long)1 return x + y")); - assertEquals(2L, exec("def x = (int)1 def y = (long)1 return x + y")); - assertEquals(2L, exec("def x = (long)1 def y = (long)1 return x + y")); - assertEquals(2F, exec("def x = (float)1 def y = (long)1 return x + y")); - assertEquals(2D, exec("def x = (double)1 def y = (long)1 return x + y")); + assertEquals(2L, exec("def x = (byte)1; def y = (long)1; return x + y")); + assertEquals(2L, exec("def x = (short)1; def y = (long)1; return x + y")); + assertEquals(2L, exec("def x = (char)1; def y = (long)1; return x + y")); + assertEquals(2L, exec("def x = (int)1; def y = (long)1; return x + y")); + assertEquals(2L, exec("def x = (long)1; def y = (long)1; return x + y")); + assertEquals(2F, exec("def x = (float)1; def y = (long)1; return x + y")); + assertEquals(2D, exec("def x = (double)1; def y = (long)1; return x + y")); - assertEquals(2F, exec("def x = (byte)1 def y = (float)1 return x + y")); - assertEquals(2F, exec("def x = (short)1 def y = (float)1 return x + y")); - assertEquals(2F, exec("def x = (char)1 def y = (float)1 return x + y")); - assertEquals(2F, exec("def x = (int)1 def y = (float)1 return x + y")); - assertEquals(2F, exec("def x = (long)1 def y = (float)1 return x + y")); - assertEquals(2F, exec("def x = (float)1 def y = (float)1 return x + y")); - assertEquals(2D, exec("def x = (double)1 def y = (float)1 return x + y")); + assertEquals(2F, exec("def x = (byte)1; def y = (float)1; return x + y")); + assertEquals(2F, exec("def x = (short)1; def y = (float)1; return x + y")); + assertEquals(2F, exec("def x = (char)1; def y = (float)1; return x + y")); + assertEquals(2F, exec("def x = (int)1; def y = (float)1; return x + y")); + assertEquals(2F, exec("def x = (long)1; def y = (float)1; return x + y")); + assertEquals(2F, exec("def x = (float)1; def y = (float)1; return x + y")); + assertEquals(2D, exec("def x = (double)1; def y = (float)1; return x + y")); - assertEquals(2D, exec("def x = (byte)1 def y = (double)1 return x + y")); - assertEquals(2D, exec("def x = (short)1 def y = (double)1 return x + y")); - assertEquals(2D, exec("def x = (char)1 def y = (double)1 return x + y")); - assertEquals(2D, exec("def x = (int)1 def y = (double)1 return x + y")); - assertEquals(2D, exec("def x = (long)1 def y = (double)1 return x + y")); - assertEquals(2D, exec("def x = (float)1 def y = (double)1 return x + y")); - assertEquals(2D, exec("def x = (double)1 def y = (double)1 return x + y")); + assertEquals(2D, exec("def x = (byte)1; def y = (double)1; return x + y")); + assertEquals(2D, exec("def x = (short)1; def y = (double)1; return x + y")); + assertEquals(2D, exec("def x = (char)1; def y = (double)1; return x + y")); + assertEquals(2D, exec("def x = (int)1; def y = (double)1; return x + y")); + assertEquals(2D, exec("def x = (long)1; def y = (double)1; return x + y")); + assertEquals(2D, exec("def x = (float)1; def y = (double)1; return x + y")); + assertEquals(2D, exec("def x = (double)1; def y = (double)1; return x + y")); - assertEquals(2, exec("def x = (Byte)1 def y = (byte)1 return x + y")); - assertEquals(2, exec("def x = (Short)1 def y = (short)1 return x + y")); - assertEquals(2, exec("def x = (Character)1 def y = (char)1 return x + y")); - assertEquals(2, exec("def x = (Integer)1 def y = (int)1 return x + y")); - assertEquals(2L, exec("def x = (Long)1 def y = (long)1 return x + y")); - assertEquals(2F, exec("def x = (Float)1 def y = (float)1 return x + y")); - assertEquals(2D, exec("def x = (Double)1 def y = (double)1 return x + y")); + assertEquals(2, exec("def x = (Byte)1; def y = (byte)1; return x + y")); + assertEquals(2, exec("def x = (Short)1; def y = (short)1; return x + y")); + assertEquals(2, exec("def x = (Character)1; def y = (char)1; return x + y")); + assertEquals(2, exec("def x = (Integer)1; def y = (int)1; return x + y")); + assertEquals(2L, exec("def x = (Long)1; def y = (long)1; return x + y")); + assertEquals(2F, exec("def x = (Float)1; def y = (float)1; return x + y")); + assertEquals(2D, exec("def x = (Double)1; def y = (double)1; return x + y")); } public void testSub() { - assertEquals(0, exec("def x = (byte)1 def y = (byte)1 return x - y")); - assertEquals(0, exec("def x = (short)1 def y = (byte)1 return x - y")); - assertEquals(0, exec("def x = (char)1 def y = (byte)1 return x - y")); - assertEquals(0, exec("def x = (int)1 def y = (byte)1 return x - y")); - assertEquals(0L, exec("def x = (long)1 def y = (byte)1 return x - y")); - assertEquals(0F, exec("def x = (float)1 def y = (byte)1 return x - y")); - assertEquals(0D, exec("def x = (double)1 def y = (byte)1 return x - y")); + assertEquals(0, exec("def x = (byte)1; def y = (byte)1; return x - y")); + assertEquals(0, exec("def x = (short)1; def y = (byte)1; return x - y")); + assertEquals(0, exec("def x = (char)1; def y = (byte)1; return x - y")); + assertEquals(0, exec("def x = (int)1; def y = (byte)1; return x - y")); + assertEquals(0L, exec("def x = (long)1; def y = (byte)1; return x - y")); + assertEquals(0F, exec("def x = (float)1; def y = (byte)1; return x - y")); + assertEquals(0D, exec("def x = (double)1; def y = (byte)1; return x - y")); - assertEquals(0, exec("def x = (byte)1 def y = (short)1 return x - y")); - assertEquals(0, exec("def x = (short)1 def y = (short)1 return x - y")); - assertEquals(0, exec("def x = (char)1 def y = (short)1 return x - y")); - assertEquals(0, exec("def x = (int)1 def y = (short)1 return x - y")); - assertEquals(0L, exec("def x = (long)1 def y = (short)1 return x - y")); - assertEquals(0F, exec("def x = (float)1 def y = (short)1 return x - y")); - assertEquals(0D, exec("def x = (double)1 def y = (short)1 return x - y")); + assertEquals(0, exec("def x = (byte)1; def y = (short)1; return x - y")); + assertEquals(0, exec("def x = (short)1; def y = (short)1; return x - y")); + assertEquals(0, exec("def x = (char)1; def y = (short)1; return x - y")); + assertEquals(0, exec("def x = (int)1; def y = (short)1; return x - y")); + assertEquals(0L, exec("def x = (long)1; def y = (short)1; return x - y")); + assertEquals(0F, exec("def x = (float)1; def y = (short)1; return x - y")); + assertEquals(0D, exec("def x = (double)1; def y = (short)1; return x - y")); - assertEquals(0, exec("def x = (byte)1 def y = (char)1 return x - y")); - assertEquals(0, exec("def x = (short)1 def y = (char)1 return x - y")); - assertEquals(0, exec("def x = (char)1 def y = (char)1 return x - y")); - assertEquals(0, exec("def x = (int)1 def y = (char)1 return x - y")); - assertEquals(0L, exec("def x = (long)1 def y = (char)1 return x - y")); - assertEquals(0F, exec("def x = (float)1 def y = (char)1 return x - y")); - assertEquals(0D, exec("def x = (double)1 def y = (char)1 return x - y")); + assertEquals(0, exec("def x = (byte)1; def y = (char)1; return x - y")); + assertEquals(0, exec("def x = (short)1; def y = (char)1; return x - y")); + assertEquals(0, exec("def x = (char)1; def y = (char)1; return x - y")); + assertEquals(0, exec("def x = (int)1; def y = (char)1; return x - y")); + assertEquals(0L, exec("def x = (long)1; def y = (char)1; return x - y")); + assertEquals(0F, exec("def x = (float)1; def y = (char)1; return x - y")); + assertEquals(0D, exec("def x = (double)1; def y = (char)1; return x - y")); - assertEquals(0, exec("def x = (byte)1 def y = (int)1 return x - y")); - assertEquals(0, exec("def x = (short)1 def y = (int)1 return x - y")); - assertEquals(0, exec("def x = (char)1 def y = (int)1 return x - y")); - assertEquals(0, exec("def x = (int)1 def y = (int)1 return x - y")); - assertEquals(0L, exec("def x = (long)1 def y = (int)1 return x - y")); - assertEquals(0F, exec("def x = (float)1 def y = (int)1 return x - y")); - assertEquals(0D, exec("def x = (double)1 def y = (int)1 return x - y")); + assertEquals(0, exec("def x = (byte)1; def y = (int)1; return x - y")); + assertEquals(0, exec("def x = (short)1; def y = (int)1; return x - y")); + assertEquals(0, exec("def x = (char)1; def y = (int)1; return x - y")); + assertEquals(0, exec("def x = (int)1; def y = (int)1; return x - y")); + assertEquals(0L, exec("def x = (long)1; def y = (int)1; return x - y")); + assertEquals(0F, exec("def x = (float)1; def y = (int)1; return x - y")); + assertEquals(0D, exec("def x = (double)1; def y = (int)1; return x - y")); - assertEquals(0L, exec("def x = (byte)1 def y = (long)1 return x - y")); - assertEquals(0L, exec("def x = (short)1 def y = (long)1 return x - y")); - assertEquals(0L, exec("def x = (char)1 def y = (long)1 return x - y")); - assertEquals(0L, exec("def x = (int)1 def y = (long)1 return x - y")); - assertEquals(0L, exec("def x = (long)1 def y = (long)1 return x - y")); - assertEquals(0F, exec("def x = (float)1 def y = (long)1 return x - y")); - assertEquals(0D, exec("def x = (double)1 def y = (long)1 return x - y")); + assertEquals(0L, exec("def x = (byte)1; def y = (long)1; return x - y")); + assertEquals(0L, exec("def x = (short)1; def y = (long)1; return x - y")); + assertEquals(0L, exec("def x = (char)1; def y = (long)1; return x - y")); + assertEquals(0L, exec("def x = (int)1; def y = (long)1; return x - y")); + assertEquals(0L, exec("def x = (long)1; def y = (long)1; return x - y")); + assertEquals(0F, exec("def x = (float)1; def y = (long)1; return x - y")); + assertEquals(0D, exec("def x = (double)1; def y = (long)1; return x - y")); - assertEquals(0F, exec("def x = (byte)1 def y = (float)1 return x - y")); - assertEquals(0F, exec("def x = (short)1 def y = (float)1 return x - y")); - assertEquals(0F, exec("def x = (char)1 def y = (float)1 return x - y")); - assertEquals(0F, exec("def x = (int)1 def y = (float)1 return x - y")); - assertEquals(0F, exec("def x = (long)1 def y = (float)1 return x - y")); - assertEquals(0F, exec("def x = (float)1 def y = (float)1 return x - y")); - assertEquals(0D, exec("def x = (double)1 def y = (float)1 return x - y")); + assertEquals(0F, exec("def x = (byte)1; def y = (float)1; return x - y")); + assertEquals(0F, exec("def x = (short)1; def y = (float)1; return x - y")); + assertEquals(0F, exec("def x = (char)1; def y = (float)1; return x - y")); + assertEquals(0F, exec("def x = (int)1; def y = (float)1; return x - y")); + assertEquals(0F, exec("def x = (long)1; def y = (float)1; return x - y")); + assertEquals(0F, exec("def x = (float)1; def y = (float)1; return x - y")); + assertEquals(0D, exec("def x = (double)1; def y = (float)1; return x - y")); - assertEquals(0D, exec("def x = (byte)1 def y = (double)1 return x - y")); - assertEquals(0D, exec("def x = (short)1 def y = (double)1 return x - y")); - assertEquals(0D, exec("def x = (char)1 def y = (double)1 return x - y")); - assertEquals(0D, exec("def x = (int)1 def y = (double)1 return x - y")); - assertEquals(0D, exec("def x = (long)1 def y = (double)1 return x - y")); - assertEquals(0D, exec("def x = (float)1 def y = (double)1 return x - y")); - assertEquals(0D, exec("def x = (double)1 def y = (double)1 return x - y")); + assertEquals(0D, exec("def x = (byte)1; def y = (double)1; return x - y")); + assertEquals(0D, exec("def x = (short)1; def y = (double)1; return x - y")); + assertEquals(0D, exec("def x = (char)1; def y = (double)1; return x - y")); + assertEquals(0D, exec("def x = (int)1; def y = (double)1; return x - y")); + assertEquals(0D, exec("def x = (long)1; def y = (double)1; return x - y")); + assertEquals(0D, exec("def x = (float)1; def y = (double)1; return x - y")); + assertEquals(0D, exec("def x = (double)1; def y = (double)1; return x - y")); - assertEquals(0, exec("def x = (Byte)1 def y = (byte)1 return x - y")); - assertEquals(0, exec("def x = (Short)1 def y = (short)1 return x - y")); - assertEquals(0, exec("def x = (Character)1 def y = (char)1 return x - y")); - assertEquals(0, exec("def x = (Integer)1 def y = (int)1 return x - y")); - assertEquals(0L, exec("def x = (Long)1 def y = (long)1 return x - y")); - assertEquals(0F, exec("def x = (Float)1 def y = (float)1 return x - y")); - assertEquals(0D, exec("def x = (Double)1 def y = (double)1 return x - y")); + assertEquals(0, exec("def x = (Byte)1; def y = (byte)1; return x - y")); + assertEquals(0, exec("def x = (Short)1; def y = (short)1; return x - y")); + assertEquals(0, exec("def x = (Character)1; def y = (char)1; return x - y")); + assertEquals(0, exec("def x = (Integer)1; def y = (int)1; return x - y")); + assertEquals(0L, exec("def x = (Long)1; def y = (long)1; return x - y")); + assertEquals(0F, exec("def x = (Float)1; def y = (float)1; return x - y")); + assertEquals(0D, exec("def x = (Double)1; def y = (double)1; return x - y")); } public void testLsh() { - assertEquals(2, exec("def x = (byte)1 def y = (byte)1 return x << y")); - assertEquals(2, exec("def x = (short)1 def y = (byte)1 return x << y")); - assertEquals(2, exec("def x = (char)1 def y = (byte)1 return x << y")); - assertEquals(2, exec("def x = (int)1 def y = (byte)1 return x << y")); - assertEquals(2L, exec("def x = (long)1 def y = (byte)1 return x << y")); - assertEquals(2L, exec("def x = (float)1 def y = (byte)1 return x << y")); - assertEquals(2L, exec("def x = (double)1 def y = (byte)1 return x << y")); + assertEquals(2, exec("def x = (byte)1; def y = (byte)1; return x << y")); + assertEquals(2, exec("def x = (short)1; def y = (byte)1; return x << y")); + assertEquals(2, exec("def x = (char)1; def y = (byte)1; return x << y")); + assertEquals(2, exec("def x = (int)1; def y = (byte)1; return x << y")); + assertEquals(2L, exec("def x = (long)1; def y = (byte)1; return x << y")); + assertEquals(2L, exec("def x = (float)1; def y = (byte)1; return x << y")); + assertEquals(2L, exec("def x = (double)1; def y = (byte)1; return x << y")); - assertEquals(2, exec("def x = (byte)1 def y = (short)1 return x << y")); - assertEquals(2, exec("def x = (short)1 def y = (short)1 return x << y")); - assertEquals(2, exec("def x = (char)1 def y = (short)1 return x << y")); - assertEquals(2, exec("def x = (int)1 def y = (short)1 return x << y")); - assertEquals(2L, exec("def x = (long)1 def y = (short)1 return x << y")); - assertEquals(2L, exec("def x = (float)1 def y = (short)1 return x << y")); - assertEquals(2L, exec("def x = (double)1 def y = (short)1 return x << y")); + assertEquals(2, exec("def x = (byte)1; def y = (short)1; return x << y")); + assertEquals(2, exec("def x = (short)1; def y = (short)1; return x << y")); + assertEquals(2, exec("def x = (char)1; def y = (short)1; return x << y")); + assertEquals(2, exec("def x = (int)1; def y = (short)1; return x << y")); + assertEquals(2L, exec("def x = (long)1; def y = (short)1; return x << y")); + assertEquals(2L, exec("def x = (float)1; def y = (short)1; return x << y")); + assertEquals(2L, exec("def x = (double)1; def y = (short)1; return x << y")); - assertEquals(2, exec("def x = (byte)1 def y = (char)1 return x << y")); - assertEquals(2, exec("def x = (short)1 def y = (char)1 return x << y")); - assertEquals(2, exec("def x = (char)1 def y = (char)1 return x << y")); - assertEquals(2, exec("def x = (int)1 def y = (char)1 return x << y")); - assertEquals(2L, exec("def x = (long)1 def y = (char)1 return x << y")); - assertEquals(2L, exec("def x = (float)1 def y = (char)1 return x << y")); - assertEquals(2L, exec("def x = (double)1 def y = (char)1 return x << y")); + assertEquals(2, exec("def x = (byte)1; def y = (char)1; return x << y")); + assertEquals(2, exec("def x = (short)1; def y = (char)1; return x << y")); + assertEquals(2, exec("def x = (char)1; def y = (char)1; return x << y")); + assertEquals(2, exec("def x = (int)1; def y = (char)1; return x << y")); + assertEquals(2L, exec("def x = (long)1; def y = (char)1; return x << y")); + assertEquals(2L, exec("def x = (float)1; def y = (char)1; return x << y")); + assertEquals(2L, exec("def x = (double)1; def y = (char)1; return x << y")); - assertEquals(2, exec("def x = (byte)1 def y = (int)1 return x << y")); - assertEquals(2, exec("def x = (short)1 def y = (int)1 return x << y")); - assertEquals(2, exec("def x = (char)1 def y = (int)1 return x << y")); - assertEquals(2, exec("def x = (int)1 def y = (int)1 return x << y")); - assertEquals(2L, exec("def x = (long)1 def y = (int)1 return x << y")); - assertEquals(2L, exec("def x = (float)1 def y = (int)1 return x << y")); - assertEquals(2L, exec("def x = (double)1 def y = (int)1 return x << y")); + assertEquals(2, exec("def x = (byte)1; def y = (int)1; return x << y")); + assertEquals(2, exec("def x = (short)1; def y = (int)1; return x << y")); + assertEquals(2, exec("def x = (char)1; def y = (int)1; return x << y")); + assertEquals(2, exec("def x = (int)1; def y = (int)1; return x << y")); + assertEquals(2L, exec("def x = (long)1; def y = (int)1; return x << y")); + assertEquals(2L, exec("def x = (float)1; def y = (int)1; return x << y")); + assertEquals(2L, exec("def x = (double)1; def y = (int)1; return x << y")); - assertEquals(2, exec("def x = (byte)1 def y = (long)1 return x << y")); - assertEquals(2, exec("def x = (short)1 def y = (long)1 return x << y")); - assertEquals(2, exec("def x = (char)1 def y = (long)1 return x << y")); - assertEquals(2, exec("def x = (int)1 def y = (long)1 return x << y")); - assertEquals(2L, exec("def x = (long)1 def y = (long)1 return x << y")); - assertEquals(2L, exec("def x = (float)1 def y = (long)1 return x << y")); - assertEquals(2L, exec("def x = (double)1 def y = (long)1 return x << y")); + assertEquals(2, exec("def x = (byte)1; def y = (long)1; return x << y")); + assertEquals(2, exec("def x = (short)1; def y = (long)1; return x << y")); + assertEquals(2, exec("def x = (char)1; def y = (long)1; return x << y")); + assertEquals(2, exec("def x = (int)1; def y = (long)1; return x << y")); + assertEquals(2L, exec("def x = (long)1; def y = (long)1; return x << y")); + assertEquals(2L, exec("def x = (float)1; def y = (long)1; return x << y")); + assertEquals(2L, exec("def x = (double)1; def y = (long)1; return x << y")); - assertEquals(2, exec("def x = (byte)1 def y = (float)1 return x << y")); - assertEquals(2, exec("def x = (short)1 def y = (float)1 return x << y")); - assertEquals(2, exec("def x = (char)1 def y = (float)1 return x << y")); - assertEquals(2, exec("def x = (int)1 def y = (float)1 return x << y")); - assertEquals(2L, exec("def x = (long)1 def y = (float)1 return x << y")); - assertEquals(2L, exec("def x = (float)1 def y = (float)1 return x << y")); - assertEquals(2L, exec("def x = (double)1 def y = (float)1 return x << y")); + assertEquals(2, exec("def x = (byte)1; def y = (float)1; return x << y")); + assertEquals(2, exec("def x = (short)1; def y = (float)1; return x << y")); + assertEquals(2, exec("def x = (char)1; def y = (float)1; return x << y")); + assertEquals(2, exec("def x = (int)1; def y = (float)1; return x << y")); + assertEquals(2L, exec("def x = (long)1; def y = (float)1; return x << y")); + assertEquals(2L, exec("def x = (float)1; def y = (float)1; return x << y")); + assertEquals(2L, exec("def x = (double)1; def y = (float)1; return x << y")); - assertEquals(2, exec("def x = (byte)1 def y = (double)1 return x << y")); - assertEquals(2, exec("def x = (short)1 def y = (double)1 return x << y")); - assertEquals(2, exec("def x = (char)1 def y = (double)1 return x << y")); - assertEquals(2, exec("def x = (int)1 def y = (double)1 return x << y")); - assertEquals(2L, exec("def x = (long)1 def y = (double)1 return x << y")); - assertEquals(2L, exec("def x = (float)1 def y = (double)1 return x << y")); - assertEquals(2L, exec("def x = (double)1 def y = (double)1 return x << y")); + assertEquals(2, exec("def x = (byte)1; def y = (double)1; return x << y")); + assertEquals(2, exec("def x = (short)1; def y = (double)1; return x << y")); + assertEquals(2, exec("def x = (char)1; def y = (double)1; return x << y")); + assertEquals(2, exec("def x = (int)1; def y = (double)1; return x << y")); + assertEquals(2L, exec("def x = (long)1; def y = (double)1; return x << y")); + assertEquals(2L, exec("def x = (float)1; def y = (double)1; return x << y")); + assertEquals(2L, exec("def x = (double)1; def y = (double)1; return x << y")); - assertEquals(2, exec("def x = (Byte)1 def y = (byte)1 return x << y")); - assertEquals(2, exec("def x = (Short)1 def y = (short)1 return x << y")); - assertEquals(2, exec("def x = (Character)1 def y = (char)1 return x << y")); - assertEquals(2, exec("def x = (Integer)1 def y = (int)1 return x << y")); - assertEquals(2L, exec("def x = (Long)1 def y = (long)1 return x << y")); - assertEquals(2L, exec("def x = (Float)1 def y = (float)1 return x << y")); - assertEquals(2L, exec("def x = (Double)1 def y = (double)1 return x << y")); + assertEquals(2, exec("def x = (Byte)1; def y = (byte)1; return x << y")); + assertEquals(2, exec("def x = (Short)1; def y = (short)1; return x << y")); + assertEquals(2, exec("def x = (Character)1; def y = (char)1; return x << y")); + assertEquals(2, exec("def x = (Integer)1; def y = (int)1; return x << y")); + assertEquals(2L, exec("def x = (Long)1; def y = (long)1; return x << y")); + assertEquals(2L, exec("def x = (Float)1; def y = (float)1; return x << y")); + assertEquals(2L, exec("def x = (Double)1; def y = (double)1; return x << y")); } public void testRsh() { - assertEquals(2, exec("def x = (byte)4 def y = (byte)1 return x >> y")); - assertEquals(2, exec("def x = (short)4 def y = (byte)1 return x >> y")); - assertEquals(2, exec("def x = (char)4 def y = (byte)1 return x >> y")); - assertEquals(2, exec("def x = (int)4 def y = (byte)1 return x >> y")); - assertEquals(2L, exec("def x = (long)4 def y = (byte)1 return x >> y")); - assertEquals(2L, exec("def x = (float)4 def y = (byte)1 return x >> y")); - assertEquals(2L, exec("def x = (double)4 def y = (byte)1 return x >> y")); + assertEquals(2, exec("def x = (byte)4; def y = (byte)1; return x >> y")); + assertEquals(2, exec("def x = (short)4; def y = (byte)1; return x >> y")); + assertEquals(2, exec("def x = (char)4; def y = (byte)1; return x >> y")); + assertEquals(2, exec("def x = (int)4; def y = (byte)1; return x >> y")); + assertEquals(2L, exec("def x = (long)4; def y = (byte)1; return x >> y")); + assertEquals(2L, exec("def x = (float)4; def y = (byte)1; return x >> y")); + assertEquals(2L, exec("def x = (double)4; def y = (byte)1; return x >> y")); - assertEquals(2, exec("def x = (byte)4 def y = (short)1 return x >> y")); - assertEquals(2, exec("def x = (short)4 def y = (short)1 return x >> y")); - assertEquals(2, exec("def x = (char)4 def y = (short)1 return x >> y")); - assertEquals(2, exec("def x = (int)4 def y = (short)1 return x >> y")); - assertEquals(2L, exec("def x = (long)4 def y = (short)1 return x >> y")); - assertEquals(2L, exec("def x = (float)4 def y = (short)1 return x >> y")); - assertEquals(2L, exec("def x = (double)4 def y = (short)1 return x >> y")); + assertEquals(2, exec("def x = (byte)4; def y = (short)1; return x >> y")); + assertEquals(2, exec("def x = (short)4; def y = (short)1; return x >> y")); + assertEquals(2, exec("def x = (char)4; def y = (short)1; return x >> y")); + assertEquals(2, exec("def x = (int)4; def y = (short)1; return x >> y")); + assertEquals(2L, exec("def x = (long)4; def y = (short)1; return x >> y")); + assertEquals(2L, exec("def x = (float)4; def y = (short)1; return x >> y")); + assertEquals(2L, exec("def x = (double)4; def y = (short)1; return x >> y")); - assertEquals(2, exec("def x = (byte)4 def y = (char)1 return x >> y")); - assertEquals(2, exec("def x = (short)4 def y = (char)1 return x >> y")); - assertEquals(2, exec("def x = (char)4 def y = (char)1 return x >> y")); - assertEquals(2, exec("def x = (int)4 def y = (char)1 return x >> y")); - assertEquals(2L, exec("def x = (long)4 def y = (char)1 return x >> y")); - assertEquals(2L, exec("def x = (float)4 def y = (char)1 return x >> y")); - assertEquals(2L, exec("def x = (double)4 def y = (char)1 return x >> y")); + assertEquals(2, exec("def x = (byte)4; def y = (char)1; return x >> y")); + assertEquals(2, exec("def x = (short)4; def y = (char)1; return x >> y")); + assertEquals(2, exec("def x = (char)4; def y = (char)1; return x >> y")); + assertEquals(2, exec("def x = (int)4; def y = (char)1; return x >> y")); + assertEquals(2L, exec("def x = (long)4; def y = (char)1; return x >> y")); + assertEquals(2L, exec("def x = (float)4; def y = (char)1; return x >> y")); + assertEquals(2L, exec("def x = (double)4; def y = (char)1; return x >> y")); - assertEquals(2, exec("def x = (byte)4 def y = (int)1 return x >> y")); - assertEquals(2, exec("def x = (short)4 def y = (int)1 return x >> y")); - assertEquals(2, exec("def x = (char)4 def y = (int)1 return x >> y")); - assertEquals(2, exec("def x = (int)4 def y = (int)1 return x >> y")); - assertEquals(2L, exec("def x = (long)4 def y = (int)1 return x >> y")); - assertEquals(2L, exec("def x = (float)4 def y = (int)1 return x >> y")); - assertEquals(2L, exec("def x = (double)4 def y = (int)1 return x >> y")); + assertEquals(2, exec("def x = (byte)4; def y = (int)1; return x >> y")); + assertEquals(2, exec("def x = (short)4; def y = (int)1; return x >> y")); + assertEquals(2, exec("def x = (char)4; def y = (int)1; return x >> y")); + assertEquals(2, exec("def x = (int)4; def y = (int)1; return x >> y")); + assertEquals(2L, exec("def x = (long)4; def y = (int)1; return x >> y")); + assertEquals(2L, exec("def x = (float)4; def y = (int)1; return x >> y")); + assertEquals(2L, exec("def x = (double)4; def y = (int)1; return x >> y")); - assertEquals(2, exec("def x = (byte)4 def y = (long)1 return x >> y")); - assertEquals(2, exec("def x = (short)4 def y = (long)1 return x >> y")); - assertEquals(2, exec("def x = (char)4 def y = (long)1 return x >> y")); - assertEquals(2, exec("def x = (int)4 def y = (long)1 return x >> y")); - assertEquals(2L, exec("def x = (long)4 def y = (long)1 return x >> y")); - assertEquals(2L, exec("def x = (float)4 def y = (long)1 return x >> y")); - assertEquals(2L, exec("def x = (double)4 def y = (long)1 return x >> y")); + assertEquals(2, exec("def x = (byte)4; def y = (long)1; return x >> y")); + assertEquals(2, exec("def x = (short)4; def y = (long)1; return x >> y")); + assertEquals(2, exec("def x = (char)4; def y = (long)1; return x >> y")); + assertEquals(2, exec("def x = (int)4; def y = (long)1; return x >> y")); + assertEquals(2L, exec("def x = (long)4; def y = (long)1; return x >> y")); + assertEquals(2L, exec("def x = (float)4; def y = (long)1; return x >> y")); + assertEquals(2L, exec("def x = (double)4; def y = (long)1; return x >> y")); - assertEquals(2, exec("def x = (byte)4 def y = (float)1 return x >> y")); - assertEquals(2, exec("def x = (short)4 def y = (float)1 return x >> y")); - assertEquals(2, exec("def x = (char)4 def y = (float)1 return x >> y")); - assertEquals(2, exec("def x = (int)4 def y = (float)1 return x >> y")); - assertEquals(2L, exec("def x = (long)4 def y = (float)1 return x >> y")); - assertEquals(2L, exec("def x = (float)4 def y = (float)1 return x >> y")); - assertEquals(2L, exec("def x = (double)4 def y = (float)1 return x >> y")); + assertEquals(2, exec("def x = (byte)4; def y = (float)1; return x >> y")); + assertEquals(2, exec("def x = (short)4; def y = (float)1; return x >> y")); + assertEquals(2, exec("def x = (char)4; def y = (float)1; return x >> y")); + assertEquals(2, exec("def x = (int)4; def y = (float)1; return x >> y")); + assertEquals(2L, exec("def x = (long)4; def y = (float)1; return x >> y")); + assertEquals(2L, exec("def x = (float)4; def y = (float)1; return x >> y")); + assertEquals(2L, exec("def x = (double)4; def y = (float)1; return x >> y")); - assertEquals(2, exec("def x = (byte)4 def y = (double)1 return x >> y")); - assertEquals(2, exec("def x = (short)4 def y = (double)1 return x >> y")); - assertEquals(2, exec("def x = (char)4 def y = (double)1 return x >> y")); - assertEquals(2, exec("def x = (int)4 def y = (double)1 return x >> y")); - assertEquals(2L, exec("def x = (long)4 def y = (double)1 return x >> y")); - assertEquals(2L, exec("def x = (float)4 def y = (double)1 return x >> y")); - assertEquals(2L, exec("def x = (double)4 def y = (double)1 return x >> y")); + assertEquals(2, exec("def x = (byte)4; def y = (double)1; return x >> y")); + assertEquals(2, exec("def x = (short)4; def y = (double)1; return x >> y")); + assertEquals(2, exec("def x = (char)4; def y = (double)1; return x >> y")); + assertEquals(2, exec("def x = (int)4; def y = (double)1; return x >> y")); + assertEquals(2L, exec("def x = (long)4; def y = (double)1; return x >> y")); + assertEquals(2L, exec("def x = (float)4; def y = (double)1; return x >> y")); + assertEquals(2L, exec("def x = (double)4; def y = (double)1; return x >> y")); - assertEquals(2, exec("def x = (Byte)4 def y = (byte)1 return x >> y")); - assertEquals(2, exec("def x = (Short)4 def y = (short)1 return x >> y")); - assertEquals(2, exec("def x = (Character)4 def y = (char)1 return x >> y")); - assertEquals(2, exec("def x = (Integer)4 def y = (int)1 return x >> y")); - assertEquals(2L, exec("def x = (Long)4 def y = (long)1 return x >> y")); - assertEquals(2L, exec("def x = (Float)4 def y = (float)1 return x >> y")); - assertEquals(2L, exec("def x = (Double)4 def y = (double)1 return x >> y")); + assertEquals(2, exec("def x = (Byte)4; def y = (byte)1; return x >> y")); + assertEquals(2, exec("def x = (Short)4; def y = (short)1; return x >> y")); + assertEquals(2, exec("def x = (Character)4; def y = (char)1; return x >> y")); + assertEquals(2, exec("def x = (Integer)4; def y = (int)1; return x >> y")); + assertEquals(2L, exec("def x = (Long)4; def y = (long)1; return x >> y")); + assertEquals(2L, exec("def x = (Float)4; def y = (float)1; return x >> y")); + assertEquals(2L, exec("def x = (Double)4; def y = (double)1; return x >> y")); } public void testUsh() { - assertEquals(2, exec("def x = (byte)4 def y = (byte)1 return x >>> y")); - assertEquals(2, exec("def x = (short)4 def y = (byte)1 return x >>> y")); - assertEquals(2, exec("def x = (char)4 def y = (byte)1 return x >>> y")); - assertEquals(2, exec("def x = (int)4 def y = (byte)1 return x >>> y")); - assertEquals(2L, exec("def x = (long)4 def y = (byte)1 return x >>> y")); - assertEquals(2L, exec("def x = (float)4 def y = (byte)1 return x >>> y")); - assertEquals(2L, exec("def x = (double)4 def y = (byte)1 return x >>> y")); + assertEquals(2, exec("def x = (byte)4; def y = (byte)1; return x >>> y")); + assertEquals(2, exec("def x = (short)4; def y = (byte)1; return x >>> y")); + assertEquals(2, exec("def x = (char)4; def y = (byte)1; return x >>> y")); + assertEquals(2, exec("def x = (int)4; def y = (byte)1; return x >>> y")); + assertEquals(2L, exec("def x = (long)4; def y = (byte)1; return x >>> y")); + assertEquals(2L, exec("def x = (float)4; def y = (byte)1; return x >>> y")); + assertEquals(2L, exec("def x = (double)4; def y = (byte)1; return x >>> y")); - assertEquals(2, exec("def x = (byte)4 def y = (short)1 return x >>> y")); - assertEquals(2, exec("def x = (short)4 def y = (short)1 return x >>> y")); - assertEquals(2, exec("def x = (char)4 def y = (short)1 return x >>> y")); - assertEquals(2, exec("def x = (int)4 def y = (short)1 return x >>> y")); - assertEquals(2L, exec("def x = (long)4 def y = (short)1 return x >>> y")); - assertEquals(2L, exec("def x = (float)4 def y = (short)1 return x >>> y")); - assertEquals(2L, exec("def x = (double)4 def y = (short)1 return x >>> y")); + assertEquals(2, exec("def x = (byte)4; def y = (short)1; return x >>> y")); + assertEquals(2, exec("def x = (short)4; def y = (short)1; return x >>> y")); + assertEquals(2, exec("def x = (char)4; def y = (short)1; return x >>> y")); + assertEquals(2, exec("def x = (int)4; def y = (short)1; return x >>> y")); + assertEquals(2L, exec("def x = (long)4; def y = (short)1; return x >>> y")); + assertEquals(2L, exec("def x = (float)4; def y = (short)1; return x >>> y")); + assertEquals(2L, exec("def x = (double)4; def y = (short)1; return x >>> y")); - assertEquals(2, exec("def x = (byte)4 def y = (char)1 return x >>> y")); - assertEquals(2, exec("def x = (short)4 def y = (char)1 return x >>> y")); - assertEquals(2, exec("def x = (char)4 def y = (char)1 return x >>> y")); - assertEquals(2, exec("def x = (int)4 def y = (char)1 return x >>> y")); - assertEquals(2L, exec("def x = (long)4 def y = (char)1 return x >>> y")); - assertEquals(2L, exec("def x = (float)4 def y = (char)1 return x >>> y")); - assertEquals(2L, exec("def x = (double)4 def y = (char)1 return x >>> y")); + assertEquals(2, exec("def x = (byte)4; def y = (char)1; return x >>> y")); + assertEquals(2, exec("def x = (short)4; def y = (char)1; return x >>> y")); + assertEquals(2, exec("def x = (char)4; def y = (char)1; return x >>> y")); + assertEquals(2, exec("def x = (int)4; def y = (char)1; return x >>> y")); + assertEquals(2L, exec("def x = (long)4; def y = (char)1; return x >>> y")); + assertEquals(2L, exec("def x = (float)4; def y = (char)1; return x >>> y")); + assertEquals(2L, exec("def x = (double)4; def y = (char)1; return x >>> y")); - assertEquals(2, exec("def x = (byte)4 def y = (int)1 return x >>> y")); - assertEquals(2, exec("def x = (short)4 def y = (int)1 return x >>> y")); - assertEquals(2, exec("def x = (char)4 def y = (int)1 return x >>> y")); - assertEquals(2, exec("def x = (int)4 def y = (int)1 return x >>> y")); - assertEquals(2L, exec("def x = (long)4 def y = (int)1 return x >>> y")); - assertEquals(2L, exec("def x = (float)4 def y = (int)1 return x >>> y")); - assertEquals(2L, exec("def x = (double)4 def y = (int)1 return x >>> y")); + assertEquals(2, exec("def x = (byte)4; def y = (int)1; return x >>> y")); + assertEquals(2, exec("def x = (short)4; def y = (int)1; return x >>> y")); + assertEquals(2, exec("def x = (char)4; def y = (int)1; return x >>> y")); + assertEquals(2, exec("def x = (int)4; def y = (int)1; return x >>> y")); + assertEquals(2L, exec("def x = (long)4; def y = (int)1; return x >>> y")); + assertEquals(2L, exec("def x = (float)4; def y = (int)1; return x >>> y")); + assertEquals(2L, exec("def x = (double)4; def y = (int)1; return x >>> y")); - assertEquals(2, exec("def x = (byte)4 def y = (long)1 return x >>> y")); - assertEquals(2, exec("def x = (short)4 def y = (long)1 return x >>> y")); - assertEquals(2, exec("def x = (char)4 def y = (long)1 return x >>> y")); - assertEquals(2, exec("def x = (int)4 def y = (long)1 return x >>> y")); - assertEquals(2L, exec("def x = (long)4 def y = (long)1 return x >>> y")); - assertEquals(2L, exec("def x = (float)4 def y = (long)1 return x >>> y")); - assertEquals(2L, exec("def x = (double)4 def y = (long)1 return x >>> y")); + assertEquals(2, exec("def x = (byte)4; def y = (long)1; return x >>> y")); + assertEquals(2, exec("def x = (short)4; def y = (long)1; return x >>> y")); + assertEquals(2, exec("def x = (char)4; def y = (long)1; return x >>> y")); + assertEquals(2, exec("def x = (int)4; def y = (long)1; return x >>> y")); + assertEquals(2L, exec("def x = (long)4; def y = (long)1; return x >>> y")); + assertEquals(2L, exec("def x = (float)4; def y = (long)1; return x >>> y")); + assertEquals(2L, exec("def x = (double)4; def y = (long)1; return x >>> y")); - assertEquals(2, exec("def x = (byte)4 def y = (float)1 return x >>> y")); - assertEquals(2, exec("def x = (short)4 def y = (float)1 return x >>> y")); - assertEquals(2, exec("def x = (char)4 def y = (float)1 return x >>> y")); - assertEquals(2, exec("def x = (int)4 def y = (float)1 return x >>> y")); - assertEquals(2L, exec("def x = (long)4 def y = (float)1 return x >>> y")); - assertEquals(2L, exec("def x = (float)4 def y = (float)1 return x >>> y")); - assertEquals(2L, exec("def x = (double)4 def y = (float)1 return x >>> y")); + assertEquals(2, exec("def x = (byte)4; def y = (float)1; return x >>> y")); + assertEquals(2, exec("def x = (short)4; def y = (float)1; return x >>> y")); + assertEquals(2, exec("def x = (char)4; def y = (float)1; return x >>> y")); + assertEquals(2, exec("def x = (int)4; def y = (float)1; return x >>> y")); + assertEquals(2L, exec("def x = (long)4; def y = (float)1; return x >>> y")); + assertEquals(2L, exec("def x = (float)4; def y = (float)1; return x >>> y")); + assertEquals(2L, exec("def x = (double)4; def y = (float)1; return x >>> y")); - assertEquals(2, exec("def x = (byte)4 def y = (double)1 return x >>> y")); - assertEquals(2, exec("def x = (short)4 def y = (double)1 return x >>> y")); - assertEquals(2, exec("def x = (char)4 def y = (double)1 return x >>> y")); - assertEquals(2, exec("def x = (int)4 def y = (double)1 return x >>> y")); - assertEquals(2L, exec("def x = (long)4 def y = (double)1 return x >>> y")); - assertEquals(2L, exec("def x = (float)4 def y = (double)1 return x >>> y")); - assertEquals(2L, exec("def x = (double)4 def y = (double)1 return x >>> y")); + assertEquals(2, exec("def x = (byte)4; def y = (double)1; return x >>> y")); + assertEquals(2, exec("def x = (short)4; def y = (double)1; return x >>> y")); + assertEquals(2, exec("def x = (char)4; def y = (double)1; return x >>> y")); + assertEquals(2, exec("def x = (int)4; def y = (double)1; return x >>> y")); + assertEquals(2L, exec("def x = (long)4; def y = (double)1; return x >>> y")); + assertEquals(2L, exec("def x = (float)4; def y = (double)1; return x >>> y")); + assertEquals(2L, exec("def x = (double)4; def y = (double)1; return x >>> y")); - assertEquals(2, exec("def x = (Byte)4 def y = (byte)1 return x >>> y")); - assertEquals(2, exec("def x = (Short)4 def y = (short)1 return x >>> y")); - assertEquals(2, exec("def x = (Character)4 def y = (char)1 return x >>> y")); - assertEquals(2, exec("def x = (Integer)4 def y = (int)1 return x >>> y")); - assertEquals(2L, exec("def x = (Long)4 def y = (long)1 return x >>> y")); - assertEquals(2L, exec("def x = (Float)4 def y = (float)1 return x >>> y")); - assertEquals(2L, exec("def x = (Double)4 def y = (double)1 return x >>> y")); + assertEquals(2, exec("def x = (Byte)4; def y = (byte)1; return x >>> y")); + assertEquals(2, exec("def x = (Short)4; def y = (short)1; return x >>> y")); + assertEquals(2, exec("def x = (Character)4; def y = (char)1; return x >>> y")); + assertEquals(2, exec("def x = (Integer)4; def y = (int)1; return x >>> y")); + assertEquals(2L, exec("def x = (Long)4; def y = (long)1; return x >>> y")); + assertEquals(2L, exec("def x = (Float)4; def y = (float)1; return x >>> y")); + assertEquals(2L, exec("def x = (Double)4; def y = (double)1; return x >>> y")); } public void testAnd() { - assertEquals(0, exec("def x = (byte)4 def y = (byte)1 return x & y")); - assertEquals(0, exec("def x = (short)4 def y = (byte)1 return x & y")); - assertEquals(0, exec("def x = (char)4 def y = (byte)1 return x & y")); - assertEquals(0, exec("def x = (int)4 def y = (byte)1 return x & y")); - assertEquals(0L, exec("def x = (long)4 def y = (byte)1 return x & y")); - assertEquals(0L, exec("def x = (float)4 def y = (byte)1 return x & y")); - assertEquals(0L, exec("def x = (double)4 def y = (byte)1 return x & y")); + assertEquals(0, exec("def x = (byte)4; def y = (byte)1; return x & y")); + assertEquals(0, exec("def x = (short)4; def y = (byte)1; return x & y")); + assertEquals(0, exec("def x = (char)4; def y = (byte)1; return x & y")); + assertEquals(0, exec("def x = (int)4; def y = (byte)1; return x & y")); + assertEquals(0L, exec("def x = (long)4; def y = (byte)1; return x & y")); + assertEquals(0L, exec("def x = (float)4; def y = (byte)1; return x & y")); + assertEquals(0L, exec("def x = (double)4; def y = (byte)1; return x & y")); - assertEquals(0, exec("def x = (byte)4 def y = (short)1 return x & y")); - assertEquals(0, exec("def x = (short)4 def y = (short)1 return x & y")); - assertEquals(0, exec("def x = (char)4 def y = (short)1 return x & y")); - assertEquals(0, exec("def x = (int)4 def y = (short)1 return x & y")); - assertEquals(0L, exec("def x = (long)4 def y = (short)1 return x & y")); - assertEquals(0L, exec("def x = (float)4 def y = (short)1 return x & y")); - assertEquals(0L, exec("def x = (double)4 def y = (short)1 return x & y")); + assertEquals(0, exec("def x = (byte)4; def y = (short)1; return x & y")); + assertEquals(0, exec("def x = (short)4; def y = (short)1; return x & y")); + assertEquals(0, exec("def x = (char)4; def y = (short)1; return x & y")); + assertEquals(0, exec("def x = (int)4; def y = (short)1; return x & y")); + assertEquals(0L, exec("def x = (long)4; def y = (short)1; return x & y")); + assertEquals(0L, exec("def x = (float)4; def y = (short)1; return x & y")); + assertEquals(0L, exec("def x = (double)4; def y = (short)1; return x & y")); - assertEquals(0, exec("def x = (byte)4 def y = (char)1 return x & y")); - assertEquals(0, exec("def x = (short)4 def y = (char)1 return x & y")); - assertEquals(0, exec("def x = (char)4 def y = (char)1 return x & y")); - assertEquals(0, exec("def x = (int)4 def y = (char)1 return x & y")); - assertEquals(0L, exec("def x = (long)4 def y = (char)1 return x & y")); - assertEquals(0L, exec("def x = (float)4 def y = (char)1 return x & y")); - assertEquals(0L, exec("def x = (double)4 def y = (char)1 return x & y")); + assertEquals(0, exec("def x = (byte)4; def y = (char)1; return x & y")); + assertEquals(0, exec("def x = (short)4; def y = (char)1; return x & y")); + assertEquals(0, exec("def x = (char)4; def y = (char)1; return x & y")); + assertEquals(0, exec("def x = (int)4; def y = (char)1; return x & y")); + assertEquals(0L, exec("def x = (long)4; def y = (char)1; return x & y")); + assertEquals(0L, exec("def x = (float)4; def y = (char)1; return x & y")); + assertEquals(0L, exec("def x = (double)4; def y = (char)1; return x & y")); - assertEquals(0, exec("def x = (byte)4 def y = (int)1 return x & y")); - assertEquals(0, exec("def x = (short)4 def y = (int)1 return x & y")); - assertEquals(0, exec("def x = (char)4 def y = (int)1 return x & y")); - assertEquals(0, exec("def x = (int)4 def y = (int)1 return x & y")); - assertEquals(0L, exec("def x = (long)4 def y = (int)1 return x & y")); - assertEquals(0L, exec("def x = (float)4 def y = (int)1 return x & y")); - assertEquals(0L, exec("def x = (double)4 def y = (int)1 return x & y")); + assertEquals(0, exec("def x = (byte)4; def y = (int)1; return x & y")); + assertEquals(0, exec("def x = (short)4; def y = (int)1; return x & y")); + assertEquals(0, exec("def x = (char)4; def y = (int)1; return x & y")); + assertEquals(0, exec("def x = (int)4; def y = (int)1; return x & y")); + assertEquals(0L, exec("def x = (long)4; def y = (int)1; return x & y")); + assertEquals(0L, exec("def x = (float)4; def y = (int)1; return x & y")); + assertEquals(0L, exec("def x = (double)4; def y = (int)1; return x & y")); - assertEquals(0L, exec("def x = (byte)4 def y = (long)1 return x & y")); - assertEquals(0L, exec("def x = (short)4 def y = (long)1 return x & y")); - assertEquals(0L, exec("def x = (char)4 def y = (long)1 return x & y")); - assertEquals(0L, exec("def x = (int)4 def y = (long)1 return x & y")); - assertEquals(0L, exec("def x = (long)4 def y = (long)1 return x & y")); - assertEquals(0L, exec("def x = (float)4 def y = (long)1 return x & y")); - assertEquals(0L, exec("def x = (double)4 def y = (long)1 return x & y")); + assertEquals(0L, exec("def x = (byte)4; def y = (long)1; return x & y")); + assertEquals(0L, exec("def x = (short)4; def y = (long)1; return x & y")); + assertEquals(0L, exec("def x = (char)4; def y = (long)1; return x & y")); + assertEquals(0L, exec("def x = (int)4; def y = (long)1; return x & y")); + assertEquals(0L, exec("def x = (long)4; def y = (long)1; return x & y")); + assertEquals(0L, exec("def x = (float)4; def y = (long)1; return x & y")); + assertEquals(0L, exec("def x = (double)4; def y = (long)1; return x & y")); - assertEquals(0L, exec("def x = (byte)4 def y = (float)1 return x & y")); - assertEquals(0L, exec("def x = (short)4 def y = (float)1 return x & y")); - assertEquals(0L, exec("def x = (char)4 def y = (float)1 return x & y")); - assertEquals(0L, exec("def x = (int)4 def y = (float)1 return x & y")); - assertEquals(0L, exec("def x = (long)4 def y = (float)1 return x & y")); - assertEquals(0L, exec("def x = (float)4 def y = (float)1 return x & y")); - assertEquals(0L, exec("def x = (double)4 def y = (float)1 return x & y")); + assertEquals(0L, exec("def x = (byte)4; def y = (float)1; return x & y")); + assertEquals(0L, exec("def x = (short)4; def y = (float)1; return x & y")); + assertEquals(0L, exec("def x = (char)4; def y = (float)1; return x & y")); + assertEquals(0L, exec("def x = (int)4; def y = (float)1; return x & y")); + assertEquals(0L, exec("def x = (long)4; def y = (float)1; return x & y")); + assertEquals(0L, exec("def x = (float)4; def y = (float)1; return x & y")); + assertEquals(0L, exec("def x = (double)4; def y = (float)1; return x & y")); - assertEquals(0L, exec("def x = (byte)4 def y = (double)1 return x & y")); - assertEquals(0L, exec("def x = (short)4 def y = (double)1 return x & y")); - assertEquals(0L, exec("def x = (char)4 def y = (double)1 return x & y")); - assertEquals(0L, exec("def x = (int)4 def y = (double)1 return x & y")); - assertEquals(0L, exec("def x = (long)4 def y = (double)1 return x & y")); - assertEquals(0L, exec("def x = (float)4 def y = (double)1 return x & y")); - assertEquals(0L, exec("def x = (double)4 def y = (double)1 return x & y")); + assertEquals(0L, exec("def x = (byte)4; def y = (double)1; return x & y")); + assertEquals(0L, exec("def x = (short)4; def y = (double)1; return x & y")); + assertEquals(0L, exec("def x = (char)4; def y = (double)1; return x & y")); + assertEquals(0L, exec("def x = (int)4; def y = (double)1; return x & y")); + assertEquals(0L, exec("def x = (long)4; def y = (double)1; return x & y")); + assertEquals(0L, exec("def x = (float)4; def y = (double)1; return x & y")); + assertEquals(0L, exec("def x = (double)4; def y = (double)1; return x & y")); - assertEquals(0, exec("def x = (Byte)4 def y = (byte)1 return x & y")); - assertEquals(0, exec("def x = (Short)4 def y = (short)1 return x & y")); - assertEquals(0, exec("def x = (Character)4 def y = (char)1 return x & y")); - assertEquals(0, exec("def x = (Integer)4 def y = (int)1 return x & y")); - assertEquals(0L, exec("def x = (Long)4 def y = (long)1 return x & y")); - assertEquals(0L, exec("def x = (Float)4 def y = (float)1 return x & y")); - assertEquals(0L, exec("def x = (Double)4 def y = (double)1 return x & y")); + assertEquals(0, exec("def x = (Byte)4; def y = (byte)1; return x & y")); + assertEquals(0, exec("def x = (Short)4; def y = (short)1; return x & y")); + assertEquals(0, exec("def x = (Character)4; def y = (char)1; return x & y")); + assertEquals(0, exec("def x = (Integer)4; def y = (int)1; return x & y")); + assertEquals(0L, exec("def x = (Long)4; def y = (long)1; return x & y")); + assertEquals(0L, exec("def x = (Float)4; def y = (float)1; return x & y")); + assertEquals(0L, exec("def x = (Double)4; def y = (double)1; return x & y")); } public void testXor() { - assertEquals(5, exec("def x = (byte)4 def y = (byte)1 return x ^ y")); - assertEquals(5, exec("def x = (short)4 def y = (byte)1 return x ^ y")); - assertEquals(5, exec("def x = (char)4 def y = (byte)1 return x ^ y")); - assertEquals(5, exec("def x = (int)4 def y = (byte)1 return x ^ y")); - assertEquals(5L, exec("def x = (long)4 def y = (byte)1 return x ^ y")); - assertEquals(5L, exec("def x = (float)4 def y = (byte)1 return x ^ y")); - assertEquals(5L, exec("def x = (double)4 def y = (byte)1 return x ^ y")); + assertEquals(5, exec("def x = (byte)4; def y = (byte)1; return x ^ y")); + assertEquals(5, exec("def x = (short)4; def y = (byte)1; return x ^ y")); + assertEquals(5, exec("def x = (char)4; def y = (byte)1; return x ^ y")); + assertEquals(5, exec("def x = (int)4; def y = (byte)1; return x ^ y")); + assertEquals(5L, exec("def x = (long)4; def y = (byte)1; return x ^ y")); + assertEquals(5L, exec("def x = (float)4; def y = (byte)1; return x ^ y")); + assertEquals(5L, exec("def x = (double)4; def y = (byte)1; return x ^ y")); - assertEquals(5, exec("def x = (byte)4 def y = (short)1 return x ^ y")); - assertEquals(5, exec("def x = (short)4 def y = (short)1 return x ^ y")); - assertEquals(5, exec("def x = (char)4 def y = (short)1 return x ^ y")); - assertEquals(5, exec("def x = (int)4 def y = (short)1 return x ^ y")); - assertEquals(5L, exec("def x = (long)4 def y = (short)1 return x ^ y")); - assertEquals(5L, exec("def x = (float)4 def y = (short)1 return x ^ y")); - assertEquals(5L, exec("def x = (double)4 def y = (short)1 return x ^ y")); + assertEquals(5, exec("def x = (byte)4; def y = (short)1; return x ^ y")); + assertEquals(5, exec("def x = (short)4; def y = (short)1; return x ^ y")); + assertEquals(5, exec("def x = (char)4; def y = (short)1; return x ^ y")); + assertEquals(5, exec("def x = (int)4; def y = (short)1; return x ^ y")); + assertEquals(5L, exec("def x = (long)4; def y = (short)1; return x ^ y")); + assertEquals(5L, exec("def x = (float)4; def y = (short)1; return x ^ y")); + assertEquals(5L, exec("def x = (double)4; def y = (short)1; return x ^ y")); - assertEquals(5, exec("def x = (byte)4 def y = (char)1 return x ^ y")); - assertEquals(5, exec("def x = (short)4 def y = (char)1 return x ^ y")); - assertEquals(5, exec("def x = (char)4 def y = (char)1 return x ^ y")); - assertEquals(5, exec("def x = (int)4 def y = (char)1 return x ^ y")); - assertEquals(5L, exec("def x = (long)4 def y = (char)1 return x ^ y")); - assertEquals(5L, exec("def x = (float)4 def y = (char)1 return x ^ y")); - assertEquals(5L, exec("def x = (double)4 def y = (char)1 return x ^ y")); + assertEquals(5, exec("def x = (byte)4; def y = (char)1; return x ^ y")); + assertEquals(5, exec("def x = (short)4; def y = (char)1; return x ^ y")); + assertEquals(5, exec("def x = (char)4; def y = (char)1; return x ^ y")); + assertEquals(5, exec("def x = (int)4; def y = (char)1; return x ^ y")); + assertEquals(5L, exec("def x = (long)4; def y = (char)1; return x ^ y")); + assertEquals(5L, exec("def x = (float)4; def y = (char)1; return x ^ y")); + assertEquals(5L, exec("def x = (double)4; def y = (char)1; return x ^ y")); - assertEquals(5, exec("def x = (byte)4 def y = (int)1 return x ^ y")); - assertEquals(5, exec("def x = (short)4 def y = (int)1 return x ^ y")); - assertEquals(5, exec("def x = (char)4 def y = (int)1 return x ^ y")); - assertEquals(5, exec("def x = (int)4 def y = (int)1 return x ^ y")); - assertEquals(5L, exec("def x = (long)4 def y = (int)1 return x ^ y")); - assertEquals(5L, exec("def x = (float)4 def y = (int)1 return x ^ y")); - assertEquals(5L, exec("def x = (double)4 def y = (int)1 return x ^ y")); + assertEquals(5, exec("def x = (byte)4; def y = (int)1; return x ^ y")); + assertEquals(5, exec("def x = (short)4; def y = (int)1; return x ^ y")); + assertEquals(5, exec("def x = (char)4; def y = (int)1; return x ^ y")); + assertEquals(5, exec("def x = (int)4; def y = (int)1; return x ^ y")); + assertEquals(5L, exec("def x = (long)4; def y = (int)1; return x ^ y")); + assertEquals(5L, exec("def x = (float)4; def y = (int)1; return x ^ y")); + assertEquals(5L, exec("def x = (double)4; def y = (int)1; return x ^ y")); - assertEquals(5L, exec("def x = (byte)4 def y = (long)1 return x ^ y")); - assertEquals(5L, exec("def x = (short)4 def y = (long)1 return x ^ y")); - assertEquals(5L, exec("def x = (char)4 def y = (long)1 return x ^ y")); - assertEquals(5L, exec("def x = (int)4 def y = (long)1 return x ^ y")); - assertEquals(5L, exec("def x = (long)4 def y = (long)1 return x ^ y")); - assertEquals(5L, exec("def x = (float)4 def y = (long)1 return x ^ y")); - assertEquals(5L, exec("def x = (double)4 def y = (long)1 return x ^ y")); + assertEquals(5L, exec("def x = (byte)4; def y = (long)1; return x ^ y")); + assertEquals(5L, exec("def x = (short)4; def y = (long)1; return x ^ y")); + assertEquals(5L, exec("def x = (char)4; def y = (long)1; return x ^ y")); + assertEquals(5L, exec("def x = (int)4; def y = (long)1; return x ^ y")); + assertEquals(5L, exec("def x = (long)4; def y = (long)1; return x ^ y")); + assertEquals(5L, exec("def x = (float)4; def y = (long)1; return x ^ y")); + assertEquals(5L, exec("def x = (double)4; def y = (long)1; return x ^ y")); - assertEquals(5L, exec("def x = (byte)4 def y = (float)1 return x ^ y")); - assertEquals(5L, exec("def x = (short)4 def y = (float)1 return x ^ y")); - assertEquals(5L, exec("def x = (char)4 def y = (float)1 return x ^ y")); - assertEquals(5L, exec("def x = (int)4 def y = (float)1 return x ^ y")); - assertEquals(5L, exec("def x = (long)4 def y = (float)1 return x ^ y")); - assertEquals(5L, exec("def x = (float)4 def y = (float)1 return x ^ y")); - assertEquals(5L, exec("def x = (double)4 def y = (float)1 return x ^ y")); + assertEquals(5L, exec("def x = (byte)4; def y = (float)1; return x ^ y")); + assertEquals(5L, exec("def x = (short)4; def y = (float)1; return x ^ y")); + assertEquals(5L, exec("def x = (char)4; def y = (float)1; return x ^ y")); + assertEquals(5L, exec("def x = (int)4; def y = (float)1; return x ^ y")); + assertEquals(5L, exec("def x = (long)4; def y = (float)1; return x ^ y")); + assertEquals(5L, exec("def x = (float)4; def y = (float)1; return x ^ y")); + assertEquals(5L, exec("def x = (double)4; def y = (float)1; return x ^ y")); - assertEquals(5L, exec("def x = (byte)4 def y = (double)1 return x ^ y")); - assertEquals(5L, exec("def x = (short)4 def y = (double)1 return x ^ y")); - assertEquals(5L, exec("def x = (char)4 def y = (double)1 return x ^ y")); - assertEquals(5L, exec("def x = (int)4 def y = (double)1 return x ^ y")); - assertEquals(5L, exec("def x = (long)4 def y = (double)1 return x ^ y")); - assertEquals(5L, exec("def x = (float)4 def y = (double)1 return x ^ y")); - assertEquals(5L, exec("def x = (double)4 def y = (double)1 return x ^ y")); + assertEquals(5L, exec("def x = (byte)4; def y = (double)1; return x ^ y")); + assertEquals(5L, exec("def x = (short)4; def y = (double)1; return x ^ y")); + assertEquals(5L, exec("def x = (char)4; def y = (double)1; return x ^ y")); + assertEquals(5L, exec("def x = (int)4; def y = (double)1; return x ^ y")); + assertEquals(5L, exec("def x = (long)4; def y = (double)1; return x ^ y")); + assertEquals(5L, exec("def x = (float)4; def y = (double)1; return x ^ y")); + assertEquals(5L, exec("def x = (double)4; def y = (double)1; return x ^ y")); - assertEquals(5, exec("def x = (Byte)4 def y = (byte)1 return x ^ y")); - assertEquals(5, exec("def x = (Short)4 def y = (short)1 return x ^ y")); - assertEquals(5, exec("def x = (Character)4 def y = (char)1 return x ^ y")); - assertEquals(5, exec("def x = (Integer)4 def y = (int)1 return x ^ y")); - assertEquals(5L, exec("def x = (Long)4 def y = (long)1 return x ^ y")); - assertEquals(5L, exec("def x = (Float)4 def y = (float)1 return x ^ y")); - assertEquals(5L, exec("def x = (Double)4 def y = (double)1 return x ^ y")); + assertEquals(5, exec("def x = (Byte)4; def y = (byte)1; return x ^ y")); + assertEquals(5, exec("def x = (Short)4; def y = (short)1; return x ^ y")); + assertEquals(5, exec("def x = (Character)4; def y = (char)1; return x ^ y")); + assertEquals(5, exec("def x = (Integer)4; def y = (int)1; return x ^ y")); + assertEquals(5L, exec("def x = (Long)4; def y = (long)1; return x ^ y")); + assertEquals(5L, exec("def x = (Float)4; def y = (float)1; return x ^ y")); + assertEquals(5L, exec("def x = (Double)4; def y = (double)1; return x ^ y")); } public void testOr() { - assertEquals(5, exec("def x = (byte)4 def y = (byte)1 return x | y")); - assertEquals(5, exec("def x = (short)4 def y = (byte)1 return x | y")); - assertEquals(5, exec("def x = (char)4 def y = (byte)1 return x | y")); - assertEquals(5, exec("def x = (int)4 def y = (byte)1 return x | y")); - assertEquals(5L, exec("def x = (long)4 def y = (byte)1 return x | y")); - assertEquals(5L, exec("def x = (float)4 def y = (byte)1 return x | y")); - assertEquals(5L, exec("def x = (double)4 def y = (byte)1 return x | y")); + assertEquals(5, exec("def x = (byte)4; def y = (byte)1; return x | y")); + assertEquals(5, exec("def x = (short)4; def y = (byte)1; return x | y")); + assertEquals(5, exec("def x = (char)4; def y = (byte)1; return x | y")); + assertEquals(5, exec("def x = (int)4; def y = (byte)1; return x | y")); + assertEquals(5L, exec("def x = (long)4; def y = (byte)1; return x | y")); + assertEquals(5L, exec("def x = (float)4; def y = (byte)1; return x | y")); + assertEquals(5L, exec("def x = (double)4; def y = (byte)1; return x | y")); - assertEquals(5, exec("def x = (byte)4 def y = (short)1 return x | y")); - assertEquals(5, exec("def x = (short)4 def y = (short)1 return x | y")); - assertEquals(5, exec("def x = (char)4 def y = (short)1 return x | y")); - assertEquals(5, exec("def x = (int)4 def y = (short)1 return x | y")); - assertEquals(5L, exec("def x = (long)4 def y = (short)1 return x | y")); - assertEquals(5L, exec("def x = (float)4 def y = (short)1 return x | y")); - assertEquals(5L, exec("def x = (double)4 def y = (short)1 return x | y")); + assertEquals(5, exec("def x = (byte)4; def y = (short)1; return x | y")); + assertEquals(5, exec("def x = (short)4; def y = (short)1; return x | y")); + assertEquals(5, exec("def x = (char)4; def y = (short)1; return x | y")); + assertEquals(5, exec("def x = (int)4; def y = (short)1; return x | y")); + assertEquals(5L, exec("def x = (long)4; def y = (short)1; return x | y")); + assertEquals(5L, exec("def x = (float)4; def y = (short)1; return x | y")); + assertEquals(5L, exec("def x = (double)4; def y = (short)1; return x | y")); - assertEquals(5, exec("def x = (byte)4 def y = (char)1 return x | y")); - assertEquals(5, exec("def x = (short)4 def y = (char)1 return x | y")); - assertEquals(5, exec("def x = (char)4 def y = (char)1 return x | y")); - assertEquals(5, exec("def x = (int)4 def y = (char)1 return x | y")); - assertEquals(5L, exec("def x = (long)4 def y = (char)1 return x | y")); - assertEquals(5L, exec("def x = (float)4 def y = (char)1 return x | y")); - assertEquals(5L, exec("def x = (double)4 def y = (char)1 return x | y")); + assertEquals(5, exec("def x = (byte)4; def y = (char)1; return x | y")); + assertEquals(5, exec("def x = (short)4; def y = (char)1; return x | y")); + assertEquals(5, exec("def x = (char)4; def y = (char)1; return x | y")); + assertEquals(5, exec("def x = (int)4; def y = (char)1; return x | y")); + assertEquals(5L, exec("def x = (long)4; def y = (char)1; return x | y")); + assertEquals(5L, exec("def x = (float)4; def y = (char)1; return x | y")); + assertEquals(5L, exec("def x = (double)4; def y = (char)1; return x | y")); - assertEquals(5, exec("def x = (byte)4 def y = (int)1 return x | y")); - assertEquals(5, exec("def x = (short)4 def y = (int)1 return x | y")); - assertEquals(5, exec("def x = (char)4 def y = (int)1 return x | y")); - assertEquals(5, exec("def x = (int)4 def y = (int)1 return x | y")); - assertEquals(5L, exec("def x = (long)4 def y = (int)1 return x | y")); - assertEquals(5L, exec("def x = (float)4 def y = (int)1 return x | y")); - assertEquals(5L, exec("def x = (double)4 def y = (int)1 return x | y")); + assertEquals(5, exec("def x = (byte)4; def y = (int)1; return x | y")); + assertEquals(5, exec("def x = (short)4; def y = (int)1; return x | y")); + assertEquals(5, exec("def x = (char)4; def y = (int)1; return x | y")); + assertEquals(5, exec("def x = (int)4; def y = (int)1; return x | y")); + assertEquals(5L, exec("def x = (long)4; def y = (int)1; return x | y")); + assertEquals(5L, exec("def x = (float)4; def y = (int)1; return x | y")); + assertEquals(5L, exec("def x = (double)4; def y = (int)1; return x | y")); - assertEquals(5L, exec("def x = (byte)4 def y = (long)1 return x | y")); - assertEquals(5L, exec("def x = (short)4 def y = (long)1 return x | y")); - assertEquals(5L, exec("def x = (char)4 def y = (long)1 return x | y")); - assertEquals(5L, exec("def x = (int)4 def y = (long)1 return x | y")); - assertEquals(5L, exec("def x = (long)4 def y = (long)1 return x | y")); - assertEquals(5L, exec("def x = (float)4 def y = (long)1 return x | y")); - assertEquals(5L, exec("def x = (double)4 def y = (long)1 return x | y")); + assertEquals(5L, exec("def x = (byte)4; def y = (long)1; return x | y")); + assertEquals(5L, exec("def x = (short)4; def y = (long)1; return x | y")); + assertEquals(5L, exec("def x = (char)4; def y = (long)1; return x | y")); + assertEquals(5L, exec("def x = (int)4; def y = (long)1; return x | y")); + assertEquals(5L, exec("def x = (long)4; def y = (long)1; return x | y")); + assertEquals(5L, exec("def x = (float)4; def y = (long)1; return x | y")); + assertEquals(5L, exec("def x = (double)4; def y = (long)1; return x | y")); - assertEquals(5L, exec("def x = (byte)4 def y = (float)1 return x | y")); - assertEquals(5L, exec("def x = (short)4 def y = (float)1 return x | y")); - assertEquals(5L, exec("def x = (char)4 def y = (float)1 return x | y")); - assertEquals(5L, exec("def x = (int)4 def y = (float)1 return x | y")); - assertEquals(5L, exec("def x = (long)4 def y = (float)1 return x | y")); - assertEquals(5L, exec("def x = (float)4 def y = (float)1 return x | y")); - assertEquals(5L, exec("def x = (double)4 def y = (float)1 return x | y")); + assertEquals(5L, exec("def x = (byte)4; def y = (float)1; return x | y")); + assertEquals(5L, exec("def x = (short)4; def y = (float)1; return x | y")); + assertEquals(5L, exec("def x = (char)4; def y = (float)1; return x | y")); + assertEquals(5L, exec("def x = (int)4; def y = (float)1; return x | y")); + assertEquals(5L, exec("def x = (long)4; def y = (float)1; return x | y")); + assertEquals(5L, exec("def x = (float)4; def y = (float)1; return x | y")); + assertEquals(5L, exec("def x = (double)4; def y = (float)1; return x | y")); - assertEquals(5L, exec("def x = (byte)4 def y = (double)1 return x | y")); - assertEquals(5L, exec("def x = (short)4 def y = (double)1 return x | y")); - assertEquals(5L, exec("def x = (char)4 def y = (double)1 return x | y")); - assertEquals(5L, exec("def x = (int)4 def y = (double)1 return x | y")); - assertEquals(5L, exec("def x = (long)4 def y = (double)1 return x | y")); - assertEquals(5L, exec("def x = (float)4 def y = (double)1 return x | y")); - assertEquals(5L, exec("def x = (double)4 def y = (double)1 return x | y")); + assertEquals(5L, exec("def x = (byte)4; def y = (double)1; return x | y")); + assertEquals(5L, exec("def x = (short)4; def y = (double)1; return x | y")); + assertEquals(5L, exec("def x = (char)4; def y = (double)1; return x | y")); + assertEquals(5L, exec("def x = (int)4; def y = (double)1; return x | y")); + assertEquals(5L, exec("def x = (long)4; def y = (double)1; return x | y")); + assertEquals(5L, exec("def x = (float)4; def y = (double)1; return x | y")); + assertEquals(5L, exec("def x = (double)4; def y = (double)1; return x | y")); - assertEquals(5, exec("def x = (Byte)4 def y = (byte)1 return x | y")); - assertEquals(5, exec("def x = (Short)4 def y = (short)1 return x | y")); - assertEquals(5, exec("def x = (Character)4 def y = (char)1 return x | y")); - assertEquals(5, exec("def x = (Integer)4 def y = (int)1 return x | y")); - assertEquals(5L, exec("def x = (Long)4 def y = (long)1 return x | y")); - assertEquals(5L, exec("def x = (Float)4 def y = (float)1 return x | y")); - assertEquals(5L, exec("def x = (Double)4 def y = (double)1 return x | y")); + assertEquals(5, exec("def x = (Byte)4; def y = (byte)1; return x | y")); + assertEquals(5, exec("def x = (Short)4; def y = (short)1; return x | y")); + assertEquals(5, exec("def x = (Character)4; def y = (char)1; return x | y")); + assertEquals(5, exec("def x = (Integer)4; def y = (int)1; return x | y")); + assertEquals(5L, exec("def x = (Long)4; def y = (long)1; return x | y")); + assertEquals(5L, exec("def x = (Float)4; def y = (float)1; return x | y")); + assertEquals(5L, exec("def x = (Double)4; def y = (double)1; return x | y")); } public void testEq() { - assertEquals(true, exec("def x = (byte)7 def y = (int)7 return x == y")); - assertEquals(true, exec("def x = (short)6 def y = (int)6 return x == y")); - assertEquals(true, exec("def x = (char)5 def y = (int)5 return x == y")); - assertEquals(true, exec("def x = (int)4 def y = (int)4 return x == y")); - assertEquals(false, exec("def x = (long)5 def y = (int)3 return x == y")); - assertEquals(false, exec("def x = (float)6 def y = (int)2 return x == y")); - assertEquals(false, exec("def x = (double)7 def y = (int)1 return x == y")); + assertEquals(true, exec("def x = (byte)7; def y = (int)7; return x == y")); + assertEquals(true, exec("def x = (short)6; def y = (int)6; return x == y")); + assertEquals(true, exec("def x = (char)5; def y = (int)5; return x == y")); + assertEquals(true, exec("def x = (int)4; def y = (int)4; return x == y")); + assertEquals(false, exec("def x = (long)5; def y = (int)3; return x == y")); + assertEquals(false, exec("def x = (float)6; def y = (int)2; return x == y")); + assertEquals(false, exec("def x = (double)7; def y = (int)1; return x == y")); - assertEquals(true, exec("def x = (byte)7 def y = (double)7 return x == y")); - assertEquals(true, exec("def x = (short)6 def y = (double)6 return x == y")); - assertEquals(true, exec("def x = (char)5 def y = (double)5 return x == y")); - assertEquals(true, exec("def x = (int)4 def y = (double)4 return x == y")); - assertEquals(false, exec("def x = (long)5 def y = (double)3 return x == y")); - assertEquals(false, exec("def x = (float)6 def y = (double)2 return x == y")); - assertEquals(false, exec("def x = (double)7 def y = (double)1 return x == y")); + assertEquals(true, exec("def x = (byte)7; def y = (double)7; return x == y")); + assertEquals(true, exec("def x = (short)6; def y = (double)6; return x == y")); + assertEquals(true, exec("def x = (char)5; def y = (double)5; return x == y")); + assertEquals(true, exec("def x = (int)4; def y = (double)4; return x == y")); + assertEquals(false, exec("def x = (long)5; def y = (double)3; return x == y")); + assertEquals(false, exec("def x = (float)6; def y = (double)2; return x == y")); + assertEquals(false, exec("def x = (double)7; def y = (double)1; return x == y")); - assertEquals(true, exec("def x = new HashMap() def y = new HashMap() return x == y")); - assertEquals(false, exec("def x = new HashMap() x.put(3, 3) def y = new HashMap() return x == y")); - assertEquals(true, exec("def x = new HashMap() x.put(3, 3) def y = new HashMap() y.put(3, 3) return x == y")); - assertEquals(true, exec("def x = new HashMap() def y = x x.put(3, 3) y.put(3, 3) return x == y")); + assertEquals(true, exec("def x = new HashMap(); def y = new HashMap(); return x == y")); + assertEquals(false, exec("def x = new HashMap(); x.put(3, 3); def y = new HashMap(); return x == y")); + assertEquals(true, exec("def x = new HashMap(); x.put(3, 3); def y = new HashMap(); y.put(3, 3); return x == y")); + assertEquals(true, exec("def x = new HashMap(); def y = x; x.put(3, 3); y.put(3, 3); return x == y")); } public void testEqr() { - assertEquals(false, exec("def x = (byte)7 def y = (int)7 return x === y")); - assertEquals(false, exec("def x = (short)6 def y = (int)6 return x === y")); - assertEquals(false, exec("def x = (char)5 def y = (int)5 return x === y")); - assertEquals(true, exec("def x = (int)4 def y = (int)4 return x === y")); - assertEquals(false, exec("def x = (long)5 def y = (int)3 return x === y")); - assertEquals(false, exec("def x = (float)6 def y = (int)2 return x === y")); - assertEquals(false, exec("def x = (double)7 def y = (int)1 return x === y")); + assertEquals(false, exec("def x = (byte)7; def y = (int)7; return x === y")); + assertEquals(false, exec("def x = (short)6; def y = (int)6; return x === y")); + assertEquals(false, exec("def x = (char)5; def y = (int)5; return x === y")); + assertEquals(true, exec("def x = (int)4; def y = (int)4; return x === y")); + assertEquals(false, exec("def x = (long)5; def y = (int)3; return x === y")); + assertEquals(false, exec("def x = (float)6; def y = (int)2; return x === y")); + assertEquals(false, exec("def x = (double)7; def y = (int)1; return x === y")); - assertEquals(false, exec("def x = new HashMap() def y = new HashMap() return x === y")); - assertEquals(false, exec("def x = new HashMap() x.put(3, 3) def y = new HashMap() return x === y")); - assertEquals(false, exec("def x = new HashMap() x.put(3, 3) def y = new HashMap() y.put(3, 3) return x === y")); - assertEquals(true, exec("def x = new HashMap() def y = x x.put(3, 3) y.put(3, 3) return x === y")); + assertEquals(false, exec("def x = new HashMap(); def y = new HashMap(); return x === y")); + assertEquals(false, exec("def x = new HashMap(); x.put(3, 3); def y = new HashMap(); return x === y")); + assertEquals(false, exec("def x = new HashMap(); x.put(3, 3); def y = new HashMap(); y.put(3, 3); return x === y")); + assertEquals(true, exec("def x = new HashMap(); def y = x; x.put(3, 3); y.put(3, 3); return x === y")); } public void testNe() { - assertEquals(false, exec("def x = (byte)7 def y = (int)7 return x != y")); - assertEquals(false, exec("def x = (short)6 def y = (int)6 return x != y")); - assertEquals(false, exec("def x = (char)5 def y = (int)5 return x != y")); - assertEquals(false, exec("def x = (int)4 def y = (int)4 return x != y")); - assertEquals(true, exec("def x = (long)5 def y = (int)3 return x != y")); - assertEquals(true, exec("def x = (float)6 def y = (int)2 return x != y")); - assertEquals(true, exec("def x = (double)7 def y = (int)1 return x != y")); + assertEquals(false, exec("def x = (byte)7; def y = (int)7; return x != y")); + assertEquals(false, exec("def x = (short)6; def y = (int)6; return x != y")); + assertEquals(false, exec("def x = (char)5; def y = (int)5; return x != y")); + assertEquals(false, exec("def x = (int)4; def y = (int)4; return x != y")); + assertEquals(true, exec("def x = (long)5; def y = (int)3; return x != y")); + assertEquals(true, exec("def x = (float)6; def y = (int)2; return x != y")); + assertEquals(true, exec("def x = (double)7; def y = (int)1; return x != y")); - assertEquals(false, exec("def x = (byte)7 def y = (double)7 return x != y")); - assertEquals(false, exec("def x = (short)6 def y = (double)6 return x != y")); - assertEquals(false, exec("def x = (char)5 def y = (double)5 return x != y")); - assertEquals(false, exec("def x = (int)4 def y = (double)4 return x != y")); - assertEquals(true, exec("def x = (long)5 def y = (double)3 return x != y")); - assertEquals(true, exec("def x = (float)6 def y = (double)2 return x != y")); - assertEquals(true, exec("def x = (double)7 def y = (double)1 return x != y")); + assertEquals(false, exec("def x = (byte)7; def y = (double)7; return x != y")); + assertEquals(false, exec("def x = (short)6; def y = (double)6; return x != y")); + assertEquals(false, exec("def x = (char)5; def y = (double)5; return x != y")); + assertEquals(false, exec("def x = (int)4; def y = (double)4; return x != y")); + assertEquals(true, exec("def x = (long)5; def y = (double)3; return x != y")); + assertEquals(true, exec("def x = (float)6; def y = (double)2; return x != y")); + assertEquals(true, exec("def x = (double)7; def y = (double)1; return x != y")); - assertEquals(false, exec("def x = new HashMap() def y = new HashMap() return x != y")); - assertEquals(true, exec("def x = new HashMap() x.put(3, 3) def y = new HashMap() return x != y")); - assertEquals(false, exec("def x = new HashMap() x.put(3, 3) def y = new HashMap() y.put(3, 3) return x != y")); - assertEquals(false, exec("def x = new HashMap() def y = x x.put(3, 3) y.put(3, 3) return x != y")); + assertEquals(false, exec("def x = new HashMap(); def y = new HashMap(); return x != y")); + assertEquals(true, exec("def x = new HashMap(); x.put(3, 3); def y = new HashMap(); return x != y")); + assertEquals(false, exec("def x = new HashMap(); x.put(3, 3); def y = new HashMap(); y.put(3, 3); return x != y")); + assertEquals(false, exec("def x = new HashMap(); def y = x; x.put(3, 3); y.put(3, 3); return x != y")); } public void testNer() { - assertEquals(true, exec("def x = (byte)7 def y = (int)7 return x !== y")); - assertEquals(true, exec("def x = (short)6 def y = (int)6 return x !== y")); - assertEquals(true, exec("def x = (char)5 def y = (int)5 return x !== y")); - assertEquals(false, exec("def x = (int)4 def y = (int)4 return x !== y")); - assertEquals(true, exec("def x = (long)5 def y = (int)3 return x !== y")); - assertEquals(true, exec("def x = (float)6 def y = (int)2 return x !== y")); - assertEquals(true, exec("def x = (double)7 def y = (int)1 return x !== y")); + assertEquals(true, exec("def x = (byte)7; def y = (int)7; return x !== y")); + assertEquals(true, exec("def x = (short)6; def y = (int)6; return x !== y")); + assertEquals(true, exec("def x = (char)5; def y = (int)5; return x !== y")); + assertEquals(false, exec("def x = (int)4; def y = (int)4; return x !== y")); + assertEquals(true, exec("def x = (long)5; def y = (int)3; return x !== y")); + assertEquals(true, exec("def x = (float)6; def y = (int)2; return x !== y")); + assertEquals(true, exec("def x = (double)7; def y = (int)1; return x !== y")); - assertEquals(true, exec("def x = new HashMap() def y = new HashMap() return x !== y")); - assertEquals(true, exec("def x = new HashMap() x.put(3, 3) def y = new HashMap() return x !== y")); - assertEquals(true, exec("def x = new HashMap() x.put(3, 3) def y = new HashMap() y.put(3, 3) return x !== y")); - assertEquals(false, exec("def x = new HashMap() def y = x x.put(3, 3) y.put(3, 3) return x !== y")); + assertEquals(true, exec("def x = new HashMap(); def y = new HashMap(); return x !== y")); + assertEquals(true, exec("def x = new HashMap(); x.put(3, 3); def y = new HashMap(); return x !== y")); + assertEquals(true, exec("def x = new HashMap(); x.put(3, 3); def y = new HashMap(); y.put(3, 3); return x !== y")); + assertEquals(false, exec("def x = new HashMap(); def y = x; x.put(3, 3); y.put(3, 3); return x !== y")); } public void testLt() { - assertEquals(true, exec("def x = (byte)1 def y = (int)7 return x < y")); - assertEquals(true, exec("def x = (short)2 def y = (int)6 return x < y")); - assertEquals(true, exec("def x = (char)3 def y = (int)5 return x < y")); - assertEquals(false, exec("def x = (int)4 def y = (int)4 return x < y")); - assertEquals(false, exec("def x = (long)5 def y = (int)3 return x < y")); - assertEquals(false, exec("def x = (float)6 def y = (int)2 return x < y")); - assertEquals(false, exec("def x = (double)7 def y = (int)1 return x < y")); + assertEquals(true, exec("def x = (byte)1; def y = (int)7; return x < y")); + assertEquals(true, exec("def x = (short)2; def y = (int)6; return x < y")); + assertEquals(true, exec("def x = (char)3; def y = (int)5; return x < y")); + assertEquals(false, exec("def x = (int)4; def y = (int)4; return x < y")); + assertEquals(false, exec("def x = (long)5; def y = (int)3; return x < y")); + assertEquals(false, exec("def x = (float)6; def y = (int)2; return x < y")); + assertEquals(false, exec("def x = (double)7; def y = (int)1; return x < y")); - assertEquals(true, exec("def x = (byte)1 def y = (double)7 return x < y")); - assertEquals(true, exec("def x = (short)2 def y = (double)6 return x < y")); - assertEquals(true, exec("def x = (char)3 def y = (double)5 return x < y")); - assertEquals(false, exec("def x = (int)4 def y = (double)4 return x < y")); - assertEquals(false, exec("def x = (long)5 def y = (double)3 return x < y")); - assertEquals(false, exec("def x = (float)6 def y = (double)2 return x < y")); - assertEquals(false, exec("def x = (double)7 def y = (double)1 return x < y")); + assertEquals(true, exec("def x = (byte)1; def y = (double)7; return x < y")); + assertEquals(true, exec("def x = (short)2; def y = (double)6; return x < y")); + assertEquals(true, exec("def x = (char)3; def y = (double)5; return x < y")); + assertEquals(false, exec("def x = (int)4; def y = (double)4; return x < y")); + assertEquals(false, exec("def x = (long)5; def y = (double)3; return x < y")); + assertEquals(false, exec("def x = (float)6; def y = (double)2; return x < y")); + assertEquals(false, exec("def x = (double)7; def y = (double)1; return x < y")); } public void testLte() { - assertEquals(true, exec("def x = (byte)1 def y = (int)7 return x <= y")); - assertEquals(true, exec("def x = (short)2 def y = (int)6 return x <= y")); - assertEquals(true, exec("def x = (char)3 def y = (int)5 return x <= y")); - assertEquals(true, exec("def x = (int)4 def y = (int)4 return x <= y")); - assertEquals(false, exec("def x = (long)5 def y = (int)3 return x <= y")); - assertEquals(false, exec("def x = (float)6 def y = (int)2 return x <= y")); - assertEquals(false, exec("def x = (double)7 def y = (int)1 return x <= y")); + assertEquals(true, exec("def x = (byte)1; def y = (int)7; return x <= y")); + assertEquals(true, exec("def x = (short)2; def y = (int)6; return x <= y")); + assertEquals(true, exec("def x = (char)3; def y = (int)5; return x <= y")); + assertEquals(true, exec("def x = (int)4; def y = (int)4; return x <= y")); + assertEquals(false, exec("def x = (long)5; def y = (int)3; return x <= y")); + assertEquals(false, exec("def x = (float)6; def y = (int)2; return x <= y")); + assertEquals(false, exec("def x = (double)7; def y = (int)1; return x <= y")); - assertEquals(true, exec("def x = (byte)1 def y = (double)7 return x <= y")); - assertEquals(true, exec("def x = (short)2 def y = (double)6 return x <= y")); - assertEquals(true, exec("def x = (char)3 def y = (double)5 return x <= y")); - assertEquals(true, exec("def x = (int)4 def y = (double)4 return x <= y")); - assertEquals(false, exec("def x = (long)5 def y = (double)3 return x <= y")); - assertEquals(false, exec("def x = (float)6 def y = (double)2 return x <= y")); - assertEquals(false, exec("def x = (double)7 def y = (double)1 return x <= y")); + assertEquals(true, exec("def x = (byte)1; def y = (double)7; return x <= y")); + assertEquals(true, exec("def x = (short)2; def y = (double)6; return x <= y")); + assertEquals(true, exec("def x = (char)3; def y = (double)5; return x <= y")); + assertEquals(true, exec("def x = (int)4; def y = (double)4; return x <= y")); + assertEquals(false, exec("def x = (long)5; def y = (double)3; return x <= y")); + assertEquals(false, exec("def x = (float)6; def y = (double)2; return x <= y")); + assertEquals(false, exec("def x = (double)7; def y = (double)1; return x <= y")); } public void testGt() { - assertEquals(false, exec("def x = (byte)1 def y = (int)7 return x > y")); - assertEquals(false, exec("def x = (short)2 def y = (int)6 return x > y")); - assertEquals(false, exec("def x = (char)3 def y = (int)5 return x > y")); - assertEquals(false, exec("def x = (int)4 def y = (int)4 return x > y")); - assertEquals(true, exec("def x = (long)5 def y = (int)3 return x > y")); - assertEquals(true, exec("def x = (float)6 def y = (int)2 return x > y")); - assertEquals(true, exec("def x = (double)7 def y = (int)1 return x > y")); + assertEquals(false, exec("def x = (byte)1; def y = (int)7; return x > y")); + assertEquals(false, exec("def x = (short)2; def y = (int)6; return x > y")); + assertEquals(false, exec("def x = (char)3; def y = (int)5; return x > y")); + assertEquals(false, exec("def x = (int)4; def y = (int)4; return x > y")); + assertEquals(true, exec("def x = (long)5; def y = (int)3; return x > y")); + assertEquals(true, exec("def x = (float)6; def y = (int)2; return x > y")); + assertEquals(true, exec("def x = (double)7; def y = (int)1; return x > y")); - assertEquals(false, exec("def x = (byte)1 def y = (double)7 return x > y")); - assertEquals(false, exec("def x = (short)2 def y = (double)6 return x > y")); - assertEquals(false, exec("def x = (char)3 def y = (double)5 return x > y")); - assertEquals(false, exec("def x = (int)4 def y = (double)4 return x > y")); - assertEquals(true, exec("def x = (long)5 def y = (double)3 return x > y")); - assertEquals(true, exec("def x = (float)6 def y = (double)2 return x > y")); - assertEquals(true, exec("def x = (double)7 def y = (double)1 return x > y")); + assertEquals(false, exec("def x = (byte)1; def y = (double)7; return x > y")); + assertEquals(false, exec("def x = (short)2; def y = (double)6; return x > y")); + assertEquals(false, exec("def x = (char)3; def y = (double)5; return x > y")); + assertEquals(false, exec("def x = (int)4; def y = (double)4; return x > y")); + assertEquals(true, exec("def x = (long)5; def y = (double)3; return x > y")); + assertEquals(true, exec("def x = (float)6; def y = (double)2; return x > y")); + assertEquals(true, exec("def x = (double)7; def y = (double)1; return x > y")); } public void testGte() { - assertEquals(false, exec("def x = (byte)1 def y = (int)7 return x >= y")); - assertEquals(false, exec("def x = (short)2 def y = (int)6 return x >= y")); - assertEquals(false, exec("def x = (char)3 def y = (int)5 return x >= y")); - assertEquals(true, exec("def x = (int)4 def y = (int)4 return x >= y")); - assertEquals(true, exec("def x = (long)5 def y = (int)3 return x >= y")); - assertEquals(true, exec("def x = (float)6 def y = (int)2 return x >= y")); - assertEquals(true, exec("def x = (double)7 def y = (int)1 return x >= y")); + assertEquals(false, exec("def x = (byte)1; def y = (int)7; return x >= y")); + assertEquals(false, exec("def x = (short)2; def y = (int)6; return x >= y")); + assertEquals(false, exec("def x = (char)3; def y = (int)5; return x >= y")); + assertEquals(true, exec("def x = (int)4; def y = (int)4; return x >= y")); + assertEquals(true, exec("def x = (long)5; def y = (int)3; return x >= y")); + assertEquals(true, exec("def x = (float)6; def y = (int)2; return x >= y")); + assertEquals(true, exec("def x = (double)7; def y = (int)1; return x >= y")); - assertEquals(false, exec("def x = (byte)1 def y = (double)7 return x >= y")); - assertEquals(false, exec("def x = (short)2 def y = (double)6 return x >= y")); - assertEquals(false, exec("def x = (char)3 def y = (double)5 return x >= y")); - assertEquals(true, exec("def x = (int)4 def y = (double)4 return x >= y")); - assertEquals(true, exec("def x = (long)5 def y = (double)3 return x >= y")); - assertEquals(true, exec("def x = (float)6 def y = (double)2 return x >= y")); - assertEquals(true, exec("def x = (double)7 def y = (double)1 return x >= y")); + assertEquals(false, exec("def x = (byte)1; def y = (double)7; return x >= y")); + assertEquals(false, exec("def x = (short)2; def y = (double)6; return x >= y")); + assertEquals(false, exec("def x = (char)3; def y = (double)5; return x >= y")); + assertEquals(true, exec("def x = (int)4; def y = (double)4; return x >= y")); + assertEquals(true, exec("def x = (long)5; def y = (double)3; return x >= y")); + assertEquals(true, exec("def x = (float)6; def y = (double)2; return x >= y")); + assertEquals(true, exec("def x = (double)7; def y = (double)1; return x >= y")); } } diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/NoSemiColonTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/NoSemiColonTests.java index 5ca98c2e575..f2e65fc680c 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/NoSemiColonTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/NoSemiColonTests.java @@ -19,161 +19,60 @@ package org.elasticsearch.painless; +import java.util.Collections; import java.util.HashMap; import java.util.Map; public class NoSemiColonTests extends ScriptTestCase { - public void testIfStatement() { - assertEquals(1, exec("int x = 5 if (x == 5) return 1 return 0")); - assertEquals(0, exec("int x = 4 if (x == 5) return 1 else return 0")); - assertEquals(2, exec("int x = 4 if (x == 5) return 1 else if (x == 4) return 2 else return 0")); - assertEquals(1, exec("int x = 4 if (x == 5) return 1 else if (x == 4) return 1 else return 0")); - - assertEquals(3, exec( - "int x = 5\n" + - "if (x == 5) {\n" + - " int y = 2\n" + - " \n" + - " if (y == 2) {\n" + - " x = 3\n" + - " }\n" + - " \n" + - "}\n" + - "\n" + - "return x\n")); - } - - public void testWhileStatement() { - - assertEquals("aaaaaa", exec("String c = \"a\" int x while (x < 5) { ++x c += \"a\" } return c")); - - Object value = exec( - " byte[][] b = new byte[5][5] \n" + - " byte x = 0, y \n" + - " \n" + - " while (x < 5) { \n" + - " y = 0 \n" + - " \n" + - " while (y < 5) { \n" + - " b[x][y] = (byte)(x*y) \n" + - " ++y \n" + - " } \n" + - " \n" + - " ++x \n" + - " } \n" + - " \n" + - " return b \n"); - - byte[][] b = (byte[][])value; - - for (byte x = 0; x < 5; ++x) { - for (byte y = 0; y < 5; ++y) { - assertEquals(x*y, b[x][y]); - } - } - } - - public void testDoWhileStatement() { - assertEquals("aaaaaa", exec("String c = \"a\" int x do { c += \"a\"; ++x } while (x < 5) return c")); - - Object value = exec( - " long[][] l = new long[5][5] \n" + - " long x = 0, y \n" + - " \n" + - " do { \n" + - " y = 0 \n" + - " \n" + - " do { \n" + - " l[(int)x][(int)y] = x*y; \n" + - " ++y \n" + - " } while (y < 5) \n" + - " \n" + - " ++x \n" + - " } while (x < 5) \n" + - " \n" + - " return l \n"); - - long[][] l = (long[][])value; - - for (long x = 0; x < 5; ++x) { - for (long y = 0; y < 5; ++y) { - assertEquals(x*y, l[(int)x][(int)y]); - } - } - } - - public void testForStatement() { - assertEquals("aaaaaa", exec("String c = \"a\" for (int x = 0; x < 5; ++x) c += \"a\" return c")); - - Object value = exec( - " int[][] i = new int[5][5] \n" + - " for (int x = 0; x < 5; ++x) { \n" + - " for (int y = 0; y < 5; ++y) { \n" + - " i[x][y] = x*y \n" + - " } \n" + - " } \n" + - " \n" + - " return i \n"); - - int[][] i = (int[][])value; - - for (int x = 0; x < 5; ++x) { - for (int y = 0; y < 5; ++y) { - assertEquals(x*y, i[x][y]); - } - } - } - public void testDeclarationStatement() { - assertEquals((byte)2, exec("byte a = 2 return a")); - assertEquals((short)2, exec("short a = 2 return a")); - assertEquals((char)2, exec("char a = 2 return a")); - assertEquals(2, exec("int a = 2 return a")); - assertEquals(2L, exec("long a = 2 return a")); - assertEquals(2F, exec("float a = 2 return a")); - assertEquals(2.0, exec("double a = 2 return a")); - assertEquals(false, exec("boolean a = false return a")); - assertEquals("string", exec("String a = \"string\" return a")); - assertEquals(HashMap.class, exec("Map a = new HashMap() return a").getClass()); + assertEquals((byte)2, exec("byte a = 2; return a")); + assertEquals((short)2, exec("short a = 2; return a")); + assertEquals((char)2, exec("char a = 2; return a")); + assertEquals(2, exec("int a = 2; return a")); + assertEquals(2L, exec("long a = 2; return a")); + assertEquals(2F, exec("float a = 2; return a")); + assertEquals(2.0, exec("double a = 2; return a")); + assertEquals(false, exec("boolean a = false; return a")); + assertEquals("string", exec("String a = \"string\"; return a")); + assertEquals(HashMap.class, exec("Map a = new HashMap(); return a").getClass()); - assertEquals(byte[].class, exec("byte[] a = new byte[1] return a").getClass()); - assertEquals(short[].class, exec("short[] a = new short[1] return a").getClass()); - assertEquals(char[].class, exec("char[] a = new char[1] return a").getClass()); - assertEquals(int[].class, exec("int[] a = new int[1] return a").getClass()); - assertEquals(long[].class, exec("long[] a = new long[1] return a").getClass()); - assertEquals(float[].class, exec("float[] a = new float[1] return a").getClass()); - assertEquals(double[].class, exec("double[] a = new double[1] return a").getClass()); - assertEquals(boolean[].class, exec("boolean[] a = new boolean[1] return a").getClass()); - assertEquals(String[].class, exec("String[] a = new String[1] return a").getClass()); - assertEquals(Map[].class, exec("Map[] a = new Map[1] return a").getClass()); + assertEquals(byte[].class, exec("byte[] a = new byte[1]; return a").getClass()); + assertEquals(short[].class, exec("short[] a = new short[1]; return a").getClass()); + assertEquals(char[].class, exec("char[] a = new char[1]; return a").getClass()); + assertEquals(int[].class, exec("int[] a = new int[1]; return a").getClass()); + assertEquals(long[].class, exec("long[] a = new long[1]; return a").getClass()); + assertEquals(float[].class, exec("float[] a = new float[1]; return a").getClass()); + assertEquals(double[].class, exec("double[] a = new double[1]; return a").getClass()); + assertEquals(boolean[].class, exec("boolean[] a = new boolean[1]; return a").getClass()); + assertEquals(String[].class, exec("String[] a = new String[1]; return a").getClass()); + assertEquals(Map[].class, exec("Map[] a = new Map[1]; return a").getClass()); - assertEquals(byte[][].class, exec("byte[][] a = new byte[1][2] return a").getClass()); - assertEquals(short[][][].class, exec("short[][][] a = new short[1][2][3] return a").getClass()); - assertEquals(char[][][][].class, exec("char[][][][] a = new char[1][2][3][4] return a").getClass()); - assertEquals(int[][][][][].class, exec("int[][][][][] a = new int[1][2][3][4][5] return a").getClass()); - assertEquals(long[][].class, exec("long[][] a = new long[1][2] return a").getClass()); - assertEquals(float[][][].class, exec("float[][][] a = new float[1][2][3] return a").getClass()); - assertEquals(double[][][][].class, exec("double[][][][] a = new double[1][2][3][4] return a").getClass()); - assertEquals(boolean[][][][][].class, exec("boolean[][][][][] a = new boolean[1][2][3][4][5] return a").getClass()); - assertEquals(String[][].class, exec("String[][] a = new String[1][2] return a").getClass()); - assertEquals(Map[][][].class, exec("Map[][][] a = new Map[1][2][3] return a").getClass()); + assertEquals(byte[][].class, exec("byte[][] a = new byte[1][2]; return a").getClass()); + assertEquals(short[][][].class, exec("short[][][] a = new short[1][2][3]; return a").getClass()); + assertEquals(char[][][][].class, exec("char[][][][] a = new char[1][2][3][4]; return a").getClass()); + assertEquals(int[][][][][].class, exec("int[][][][][] a = new int[1][2][3][4][5]; return a").getClass()); + assertEquals(long[][].class, exec("long[][] a = new long[1][2]; return a").getClass()); + assertEquals(float[][][].class, exec("float[][][] a = new float[1][2][3]; return a").getClass()); + assertEquals(double[][][][].class, exec("double[][][][] a = new double[1][2][3][4]; return a").getClass()); + assertEquals(boolean[][][][][].class, exec("boolean[][][][][] a = new boolean[1][2][3][4][5]; return a").getClass()); + assertEquals(String[][].class, exec("String[][] a = new String[1][2]; return a").getClass()); + assertEquals(Map[][][].class, exec("Map[][][] a = new Map[1][2][3]; return a").getClass()); } - - public void testContinueStatement() { - assertEquals(9, exec("int x = 0, y = 0 while (x < 10) { ++x if (x == 1) continue ++y } return y")); - } - - public void testBreakStatement() { - assertEquals(4, exec("int x = 0, y = 0 while (x < 10) { ++x if (x == 5) break ++y } return y")); + + public void testExpression() { + assertEquals(10, exec("10")); + assertEquals(10, exec("5 + 5")); + assertEquals(10, exec("5 + 5")); + assertEquals(10, exec("params.param == 'yes' ? 10 : 5", Collections.singletonMap("param", "yes"))); } @SuppressWarnings("rawtypes") public void testReturnStatement() { assertEquals(10, exec("return 10")); - assertEquals(5, exec("int x = 5 return x")); - assertEquals(4, exec("int[] x = new int[2] x[1] = 4 return x[1]")); - assertEquals(5, ((short[])exec("short[] s = new short[3] s[1] = 5 return s"))[1]); - assertEquals(10, ((Map)exec("Map s = new HashMap< String,Object>() s.put(\"x\", 10) return s")).get("x")); + assertEquals(5, exec("int x = 5; return x")); + assertEquals(4, exec("int[] x = new int[2]; x[1] = 4; return x[1]")); + assertEquals(5, ((short[])exec("short[] s = new short[3]; s[1] = 5; return s"))[1]); + assertEquals(10, ((Map)exec("Map s = new HashMap< String,Object>(); s.put(\"x\", 10); return s")).get("x")); } } diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/OverloadTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/OverloadTests.java new file mode 100644 index 00000000000..3e0e4a5fc71 --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/OverloadTests.java @@ -0,0 +1,52 @@ +package org.elasticsearch.painless; + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** Tests method overloading */ +public class OverloadTests extends ScriptTestCase { + + public void testMethod() { + assertEquals(2, exec("return 'abc123abc'.indexOf('c');")); + assertEquals(8, exec("return 'abc123abc'.indexOf('c', 3);")); + IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> { + exec("return 'abc123abc'.indexOf('c', 3, 'bogus');"); + }); + assertTrue(expected.getMessage().contains("[indexOf] with [3] arguments")); + } + + public void testMethodDynamic() { + assertEquals(2, exec("def x = 'abc123abc'; return x.indexOf('c');")); + assertEquals(8, exec("def x = 'abc123abc'; return x.indexOf('c', 3);")); + IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> { + exec("def x = 'abc123abc'; return x.indexOf('c', 3, 'bogus');"); + }); + assertTrue(expected.getMessage().contains("dynamic method [indexOf] with signature [(String,int,String)")); + } + + public void testConstructor() { + assertEquals(true, exec("FeatureTest f = new FeatureTest(); return f.x == 0 && f.y == 0;")); + assertEquals(true, exec("FeatureTest f = new FeatureTest(1, 2); return f.x == 1 && f.y == 2;")); + } + + public void testStatic() { + assertEquals(true, exec("return FeatureTest.overloadedStatic();")); + assertEquals(false, exec("return FeatureTest.overloadedStatic(false);")); + } +} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/StringTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/StringTests.java index 0c2b2f4386c..95acf1826fd 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/StringTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/StringTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.painless; +import java.util.Locale; + public class StringTests extends ScriptTestCase { public void testAppend() { @@ -63,6 +65,21 @@ public class StringTests extends ScriptTestCase { assertEquals("cat" + "cat", exec("String s = 'cat'; return s + s;")); } + public void testAppendMultiple() { + assertEquals("cat" + true + "abc" + null, exec("String s = \"cat\"; return s + true + 'abc' + null;")); + } + + public void testAppendMany() { + StringBuilder script = new StringBuilder("String s = \"cat\"; return s"); + StringBuilder result = new StringBuilder("cat"); + for (int i = 0; i < 200 /* indy limit */ + 10; i++) { + final String s = String.format(Locale.ROOT, "%03d", i); + script.append(" + '").append(s).append("'.toString()"); + result.append(s); + } + assertEquals(result.toString(), exec(script.toString())); + } + public void testStringAPI() { assertEquals("", exec("return new String();")); assertEquals('x', exec("String s = \"x\"; return s.charAt(0);")); @@ -127,8 +144,8 @@ public class StringTests extends ScriptTestCase { assertEquals("c", exec("return (String)(char)\"c\"")); assertEquals("c", exec("return (String)(char)'c'")); - assertEquals('c', exec("String s = \"c\" (char)s")); - assertEquals('c', exec("String s = 'c' (char)s")); + assertEquals('c', exec("String s = \"c\"; (char)s")); + assertEquals('c', exec("String s = 'c'; (char)s")); try { assertEquals("cc", exec("return (String)(char)\"cc\"")); @@ -145,14 +162,14 @@ public class StringTests extends ScriptTestCase { } try { - assertEquals('c', exec("String s = \"cc\" (char)s")); + assertEquals('c', exec("String s = \"cc\"; (char)s")); fail(); } catch (final ClassCastException cce) { assertTrue(cce.getMessage().contains("Cannot cast [String] with length greater than one to [char].")); } try { - assertEquals('c', exec("String s = 'cc' (char)s")); + assertEquals('c', exec("String s = 'cc'; (char)s")); fail(); } catch (final ClassCastException cce) { assertTrue(cce.getMessage().contains("Cannot cast [String] with length greater than one to [char].")); @@ -163,8 +180,8 @@ public class StringTests extends ScriptTestCase { assertEquals("c", exec("return (String)(Character)\"c\"")); assertEquals("c", exec("return (String)(Character)'c'")); - assertEquals('c', exec("String s = \"c\" (Character)s")); - assertEquals('c', exec("String s = 'c' (Character)s")); + assertEquals('c', exec("String s = \"c\"; (Character)s")); + assertEquals('c', exec("String s = 'c'; (Character)s")); try { assertEquals("cc", exec("return (String)(Character)\"cc\"")); @@ -181,14 +198,14 @@ public class StringTests extends ScriptTestCase { } try { - assertEquals('c', exec("String s = \"cc\" (Character)s")); + assertEquals('c', exec("String s = \"cc\"; (Character)s")); fail(); } catch (final ClassCastException cce) { assertTrue(cce.getMessage().contains("Cannot cast [String] with length greater than one to [Character].")); } try { - assertEquals('c', exec("String s = 'cc' (Character)s")); + assertEquals('c', exec("String s = 'cc'; (Character)s")); fail(); } catch (final ClassCastException cce) { assertTrue(cce.getMessage().contains("Cannot cast [String] with length greater than one to [Character].")); diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/WhenThingsGoWrongTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/WhenThingsGoWrongTests.java index 55ee490bb3f..15ae321534e 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/WhenThingsGoWrongTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/WhenThingsGoWrongTests.java @@ -90,7 +90,7 @@ public class WhenThingsGoWrongTests extends ScriptTestCase { "The maximum number of statements that can be executed in a loop has been reached.")); expected = expectThrows(PainlessError.class, () -> { - exec("while (true) {int y = 5}"); + exec("while (true) {int y = 5;}"); }); assertTrue(expected.getMessage().contains( "The maximum number of statements that can be executed in a loop has been reached.")); @@ -116,7 +116,7 @@ public class WhenThingsGoWrongTests extends ScriptTestCase { "The maximum number of statements that can be executed in a loop has been reached.")); expected = expectThrows(PainlessError.class, () -> { - exec("for (;;) {int x = 5}"); + exec("for (;;) {int x = 5;}"); fail("should have hit PainlessError"); }); assertTrue(expected.getMessage().contains( @@ -130,7 +130,7 @@ public class WhenThingsGoWrongTests extends ScriptTestCase { "The maximum number of statements that can be executed in a loop has been reached.")); RuntimeException parseException = expectThrows(RuntimeException.class, () -> { - exec("try { int x } catch (PainlessError error) {}"); + exec("try { int x; } catch (PainlessError error) {}"); fail("should have hit ParseException"); }); assertTrue(parseException.getMessage().contains("Not a type [PainlessError].")); diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java index efa01d78790..f8561d17acd 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.reindex; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; @@ -57,6 +58,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import static java.lang.Math.max; import static java.lang.Math.min; @@ -91,7 +93,8 @@ public abstract class AbstractAsyncBulkByScrollAction listener; - private final Retry retry; + private final BackoffPolicy backoffPolicy; + private final Retry bulkRetry; public AbstractAsyncBulkByScrollAction(BulkByScrollTask task, ESLogger logger, ParentTaskAssigningClient client, ThreadPool threadPool, Request mainRequest, SearchRequest firstSearchRequest, ActionListener listener) { @@ -102,7 +105,8 @@ public abstract class AbstractAsyncBulkByScrollAction docs); @@ -131,21 +135,14 @@ public abstract class AbstractAsyncBulkByScrollAction() { - @Override - public void onResponse(SearchResponse response) { - logger.debug("[{}] documents match query", response.getHits().getTotalHits()); - onScrollResponse(timeValueSeconds(0), response); - } - - @Override - public void onFailure(Throwable e) { - finishHim(e); - } - }); } catch (Throwable t) { finishHim(t); + return; } + searchWithRetry(listener -> client.search(firstSearchRequest, listener), (SearchResponse response) -> { + logger.debug("[{}] documents match query", response.getHits().getTotalHits()); + onScrollResponse(timeValueSeconds(0), response); + }); } /** @@ -239,7 +236,7 @@ public abstract class AbstractAsyncBulkByScrollAction() { + bulkRetry.withAsyncBackoff(client, request, new ActionListener() { @Override public void onResponse(BulkResponse response) { onBulkResponse(response); @@ -322,16 +319,8 @@ public abstract class AbstractAsyncBulkByScrollAction() { - @Override - public void onResponse(SearchResponse response) { - onScrollResponse(timeValueNanos(max(0, earliestNextBatchStartTime - System.nanoTime())), response); - } - - @Override - public void onFailure(Throwable e) { - finishHim(e); - } + searchWithRetry(listener -> client.searchScroll(request, listener), (SearchResponse response) -> { + onScrollResponse(timeValueNanos(max(0, earliestNextBatchStartTime - System.nanoTime())), response); }); } @@ -434,9 +423,9 @@ public abstract class AbstractAsyncBulkByScrollAction void searchWithRetry(Consumer> action, Consumer onResponse) { + class RetryHelper extends AbstractRunnable implements ActionListener { + private final Iterator retries = backoffPolicy.iterator(); + + @Override + public void onResponse(T response) { + onResponse.accept(response); + } + + @Override + protected void doRun() throws Exception { + action.accept(this); + } + + @Override + public void onFailure(Throwable e) { + if (ExceptionsHelper.unwrap(e, EsRejectedExecutionException.class) != null) { + if (retries.hasNext()) { + logger.trace("retrying rejected search", e); + threadPool.schedule(retries.next(), ThreadPool.Names.SAME, this); + task.countSearchRetry(); + } else { + logger.warn("giving up on search because we retried {} times without success", e, retries); + finishHim(e); + } + } else { + logger.warn("giving up on search because it failed with a non-retryable exception", e); + finishHim(e); + } + } + } + new RetryHelper().run(); + } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java index 14790e97acc..7505f490f45 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequest.java @@ -19,9 +19,6 @@ package org.elasticsearch.index.reindex; -import java.io.IOException; -import java.util.Arrays; - import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.WriteConsistencyLevel; @@ -34,6 +31,9 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; +import java.io.IOException; +import java.util.Arrays; + import static org.elasticsearch.action.ValidateActions.addValidationError; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes; diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java index c158a7669d2..bd1a63cb4e9 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByScrollRequestBuilder.java @@ -107,6 +107,23 @@ public abstract class AbstractBulkByScrollRequestBuilder< return self(); } + /** + * Initial delay after a rejection before retrying a bulk request. With the default maxRetries the total backoff for retrying rejections + * is about one minute per bulk request. Once the entire bulk request is successful the retry counter resets. + */ + public Self setRetryBackoffInitialTime(TimeValue retryBackoffInitialTime) { + request.setRetryBackoffInitialTime(retryBackoffInitialTime); + return self(); + } + + /** + * Total number of retries attempted for rejections. There is no way to ask for unlimited retries. + */ + public Self setMaxRetries(int maxRetries) { + request.setMaxRetries(maxRetries); + return self(); + } + /** * Set the throttle for this request in sub-requests per second. {@link Float#POSITIVE_INFINITY} means set no throttle and that is the * default. Throttling is done between batches, as we start the next scroll requests. That way we can increase the scroll's timeout to diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkIndexByScrollRequestBuilder.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkIndexByScrollRequestBuilder.java index fd86b488180..3fbeea078b7 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkIndexByScrollRequestBuilder.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkIndexByScrollRequestBuilder.java @@ -26,12 +26,11 @@ import org.elasticsearch.script.Script; public abstract class AbstractBulkIndexByScrollRequestBuilder< Request extends AbstractBulkIndexByScrollRequest, - Response extends BulkIndexByScrollResponse, - Self extends AbstractBulkIndexByScrollRequestBuilder> - extends AbstractBulkByScrollRequestBuilder { + Self extends AbstractBulkIndexByScrollRequestBuilder> + extends AbstractBulkByScrollRequestBuilder { protected AbstractBulkIndexByScrollRequestBuilder(ElasticsearchClient client, - Action action, SearchRequestBuilder search, Request request) { + Action action, SearchRequestBuilder search, Request request) { super(client, action, search, request); } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java index 9f4c8163b88..f062287a5de 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java @@ -58,7 +58,8 @@ public class BulkByScrollTask extends CancellableTask { private final AtomicLong noops = new AtomicLong(0); private final AtomicInteger batch = new AtomicInteger(0); private final AtomicLong versionConflicts = new AtomicLong(0); - private final AtomicLong retries = new AtomicLong(0); + private final AtomicLong bulkRetries = new AtomicLong(0); + private final AtomicLong searchRetries = new AtomicLong(0); private final AtomicLong throttledNanos = new AtomicLong(); /** * The number of requests per second to which to throttle the request that this task represents. The other variables are all AtomicXXX @@ -84,7 +85,8 @@ public class BulkByScrollTask extends CancellableTask { @Override public Status getStatus() { return new Status(total.get(), updated.get(), created.get(), deleted.get(), batch.get(), versionConflicts.get(), noops.get(), - retries.get(), timeValueNanos(throttledNanos.get()), getRequestsPerSecond(), getReasonCancelled(), throttledUntil()); + bulkRetries.get(), searchRetries.get(), timeValueNanos(throttledNanos.get()), getRequestsPerSecond(), getReasonCancelled(), + throttledUntil()); } private TimeValue throttledUntil() { @@ -133,14 +135,16 @@ public class BulkByScrollTask extends CancellableTask { private final int batches; private final long versionConflicts; private final long noops; - private final long retries; + private final long bulkRetries; + private final long searchRetries; private final TimeValue throttled; private final float requestsPerSecond; private final String reasonCancelled; private final TimeValue throttledUntil; - public Status(long total, long updated, long created, long deleted, int batches, long versionConflicts, long noops, long retries, - TimeValue throttled, float requestsPerSecond, @Nullable String reasonCancelled, TimeValue throttledUntil) { + public Status(long total, long updated, long created, long deleted, int batches, long versionConflicts, long noops, + long bulkRetries, long searchRetries, TimeValue throttled, float requestsPerSecond, @Nullable String reasonCancelled, + TimeValue throttledUntil) { this.total = checkPositive(total, "total"); this.updated = checkPositive(updated, "updated"); this.created = checkPositive(created, "created"); @@ -148,7 +152,8 @@ public class BulkByScrollTask extends CancellableTask { this.batches = checkPositive(batches, "batches"); this.versionConflicts = checkPositive(versionConflicts, "versionConflicts"); this.noops = checkPositive(noops, "noops"); - this.retries = checkPositive(retries, "retries"); + this.bulkRetries = checkPositive(bulkRetries, "bulkRetries"); + this.searchRetries = checkPositive(searchRetries, "searchRetries"); this.throttled = throttled; this.requestsPerSecond = requestsPerSecond; this.reasonCancelled = reasonCancelled; @@ -163,7 +168,8 @@ public class BulkByScrollTask extends CancellableTask { batches = in.readVInt(); versionConflicts = in.readVLong(); noops = in.readVLong(); - retries = in.readVLong(); + bulkRetries = in.readVLong(); + searchRetries = in.readVLong(); throttled = TimeValue.readTimeValue(in); requestsPerSecond = in.readFloat(); reasonCancelled = in.readOptionalString(); @@ -179,7 +185,8 @@ public class BulkByScrollTask extends CancellableTask { out.writeVInt(batches); out.writeVLong(versionConflicts); out.writeVLong(noops); - out.writeVLong(retries); + out.writeVLong(bulkRetries); + out.writeVLong(searchRetries); throttled.writeTo(out); out.writeFloat(requestsPerSecond); out.writeOptionalString(reasonCancelled); @@ -208,7 +215,11 @@ public class BulkByScrollTask extends CancellableTask { builder.field("batches", batches); builder.field("version_conflicts", versionConflicts); builder.field("noops", noops); - builder.field("retries", retries); + builder.startObject("retries"); { + builder.field("bulk", bulkRetries); + builder.field("search", searchRetries); + } + builder.endObject(); builder.timeValueField("throttled_millis", "throttled", throttled); builder.field("requests_per_second", requestsPerSecond == Float.POSITIVE_INFINITY ? "unlimited" : requestsPerSecond); if (reasonCancelled != null) { @@ -233,7 +244,7 @@ public class BulkByScrollTask extends CancellableTask { builder.append(",batches=").append(batches); builder.append(",versionConflicts=").append(versionConflicts); builder.append(",noops=").append(noops); - builder.append(",retries=").append(retries); + builder.append(",retries=").append(bulkRetries); if (reasonCancelled != null) { builder.append(",canceled=").append(reasonCancelled); } @@ -296,10 +307,17 @@ public class BulkByScrollTask extends CancellableTask { } /** - * Number of retries that had to be attempted due to rejected executions. + * Number of retries that had to be attempted due to bulk actions being rejected. */ - public long getRetries() { - return retries; + public long getBulkRetries() { + return bulkRetries; + } + + /** + * Number of retries that had to be attempted due to search actions being rejected. + */ + public long getSearchRetries() { + return searchRetries; } /** @@ -373,8 +391,12 @@ public class BulkByScrollTask extends CancellableTask { versionConflicts.incrementAndGet(); } - void countRetry() { - retries.incrementAndGet(); + void countBulkRetry() { + bulkRetries.incrementAndGet(); + } + + void countSearchRetry() { + searchRetries.incrementAndGet(); } float getRequestsPerSecond() { diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkIndexByScrollResponse.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkIndexByScrollResponse.java index 35d45f3d5d4..8e20375d9a7 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkIndexByScrollResponse.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkIndexByScrollResponse.java @@ -98,6 +98,20 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont return status.getReasonCancelled(); } + /** + * The number of times that the request had retry bulk actions. + */ + public long getBulkRetries() { + return status.getBulkRetries(); + } + + /** + * The number of times that the request had retry search actions. + */ + public long getSearchRetries() { + return status.getSearchRetries(); + } + /** * All of the indexing failures. Version conflicts are only included if the request sets abortOnVersionConflict to true (the * default). diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexRequest.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexRequest.java index d51fb7e8bc1..660815bbf52 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexRequest.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexRequest.java @@ -19,10 +19,6 @@ package org.elasticsearch.index.reindex; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; - import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; import org.elasticsearch.action.IndicesRequest; @@ -32,8 +28,11 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.uid.Versions; -import static java.util.Collections.unmodifiableList; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import static java.util.Collections.unmodifiableList; import static org.elasticsearch.action.ValidateActions.addValidationError; import static org.elasticsearch.index.VersionType.INTERNAL; diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java index 6b5cee46c97..4f814dbc49d 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java @@ -27,7 +27,7 @@ import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; public class ReindexRequestBuilder extends - AbstractBulkIndexByScrollRequestBuilder { + AbstractBulkIndexByScrollRequestBuilder { private final IndexRequestBuilder destination; public ReindexRequestBuilder(ElasticsearchClient client, diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java index 915921d6077..5f215a06679 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java @@ -19,14 +19,14 @@ package org.elasticsearch.index.reindex; -import java.util.ArrayList; -import java.util.List; - import org.elasticsearch.action.CompositeIndicesRequest; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.search.SearchRequest; +import java.util.ArrayList; +import java.util.List; + import static java.util.Collections.unmodifiableList; /** diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java index ef64f36b3db..8985f439fba 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java @@ -25,7 +25,7 @@ import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; public class UpdateByQueryRequestBuilder extends - AbstractBulkIndexByScrollRequestBuilder { + AbstractBulkIndexByScrollRequestBuilder { public UpdateByQueryRequestBuilder(ElasticsearchClient client, Action action) { diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java index 2409a99faf2..c9bd22d3552 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.reindex; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; @@ -38,6 +40,8 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollResponse; +import org.elasticsearch.action.search.ReduceSearchPhaseException; +import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchScrollRequest; @@ -98,6 +102,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.either; import static org.hamcrest.Matchers.emptyCollectionOf; import static org.hamcrest.Matchers.equalTo; @@ -122,8 +127,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { public void setupForTest() { client = new MyMockClient(new NoOpClient(getTestName())); threadPool = new ThreadPool(getTestName()); - testRequest = new DummyAbstractBulkByScrollRequest(); - firstSearchRequest = new SearchRequest().scroll(timeValueSeconds(10)); + firstSearchRequest = new SearchRequest(); + testRequest = new DummyAbstractBulkByScrollRequest(firstSearchRequest); listener = new PlainActionFuture<>(); scrollId = null; taskManager = new TaskManager(Settings.EMPTY); @@ -150,10 +155,62 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { * random scroll id so it is checked instead. */ private String scrollId() { - scrollId = randomSimpleString(random(), 1, 1000); // Empty string's get special behavior we don't want + scrollId = randomSimpleString(random(), 1, 1000); // Empty strings get special behavior we don't want return scrollId; } + public void testStartRetriesOnRejectionAndSucceeds() throws Exception { + client.searchesToReject = randomIntBetween(0, testRequest.getMaxRetries() - 1); + DummyAbstractAsyncBulkByScrollAction action = new DummyActionWithoutBackoff(); + action.start(); + assertBusy(() -> assertEquals(client.searchesToReject + 1, client.searchAttempts.get())); + if (listener.isDone()) { + Object result = listener.get(); + fail("Expected listener not to be done but it was and had " + result); + } + assertBusy(() -> assertNotNull("There should be a search attempt pending that we didn't reject", client.lastSearch.get())); + assertEquals(client.searchesToReject, testTask.getStatus().getSearchRetries()); + } + + public void testStartRetriesOnRejectionButFailsOnTooManyRejections() throws Exception { + client.searchesToReject = testRequest.getMaxRetries() + randomIntBetween(1, 100); + DummyAbstractAsyncBulkByScrollAction action = new DummyActionWithoutBackoff(); + action.start(); + assertBusy(() -> assertEquals(testRequest.getMaxRetries() + 1, client.searchAttempts.get())); + assertBusy(() -> assertTrue(listener.isDone())); + ExecutionException e = expectThrows(ExecutionException.class, () -> listener.get()); + assertThat(ExceptionsHelper.stackTrace(e), containsString(EsRejectedExecutionException.class.getSimpleName())); + assertNull("There shouldn't be a search attempt pending that we didn't reject", client.lastSearch.get()); + assertEquals(testRequest.getMaxRetries(), testTask.getStatus().getSearchRetries()); + } + + public void testStartNextScrollRetriesOnRejectionAndSucceeds() throws Exception { + client.scrollsToReject = randomIntBetween(0, testRequest.getMaxRetries() - 1); + DummyAbstractAsyncBulkByScrollAction action = new DummyActionWithoutBackoff(); + action.setScroll(scrollId()); + action.startNextScroll(0); + assertBusy(() -> assertEquals(client.scrollsToReject + 1, client.scrollAttempts.get())); + if (listener.isDone()) { + Object result = listener.get(); + fail("Expected listener not to be done but it was and had " + result); + } + assertBusy(() -> assertNotNull("There should be a scroll attempt pending that we didn't reject", client.lastScroll.get())); + assertEquals(client.scrollsToReject, testTask.getStatus().getSearchRetries()); + } + + public void testStartNextScrollRetriesOnRejectionButFailsOnTooManyRejections() throws Exception { + client.scrollsToReject = testRequest.getMaxRetries() + randomIntBetween(1, 100); + DummyAbstractAsyncBulkByScrollAction action = new DummyActionWithoutBackoff(); + action.setScroll(scrollId()); + action.startNextScroll(0); + assertBusy(() -> assertEquals(testRequest.getMaxRetries() + 1, client.scrollAttempts.get())); + assertBusy(() -> assertTrue(listener.isDone())); + ExecutionException e = expectThrows(ExecutionException.class, () -> listener.get()); + assertThat(ExceptionsHelper.stackTrace(e), containsString(EsRejectedExecutionException.class.getSimpleName())); + assertNull("There shouldn't be a scroll attempt pending that we didn't reject", client.lastScroll.get()); + assertEquals(testRequest.getMaxRetries(), testTask.getStatus().getSearchRetries()); + } + public void testScrollResponseSetsTotal() { // Default is 0, meaning unstarted assertEquals(0, testTask.getStatus().getTotal()); @@ -354,8 +411,9 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { int bulksToTry = randomIntBetween(1, 10); long retryAttempts = 0; for (int i = 0; i < bulksToTry; i++) { - retryAttempts += retryTestCase(false); - assertEquals(retryAttempts, testTask.getStatus().getRetries()); + bulkRetryTestCase(false); + retryAttempts += testRequest.getMaxRetries(); + assertEquals(retryAttempts, testTask.getStatus().getBulkRetries()); } } @@ -363,8 +421,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { * Mimicks bulk rejections. These should be retried but we fail anyway because we run out of retries. */ public void testBulkRejectionsRetryAndFailAnyway() throws Exception { - long retryAttempts = retryTestCase(true); - assertEquals(retryAttempts, testTask.getStatus().getRetries()); + bulkRetryTestCase(true); + assertEquals(testRequest.getMaxRetries(), testTask.getStatus().getBulkRetries()); } public void testPerfectlyThrottledBatchTime() { @@ -398,6 +456,9 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { DummyAbstractAsyncBulkByScrollAction action = new DummyAbstractAsyncBulkByScrollAction(); action.setScroll(scrollId()); + // Set the base for the scroll to wait - this is added to the figure we calculate below + firstSearchRequest.scroll(timeValueSeconds(10)); + // We'd like to get about 1 request a second testTask.rethrottle(1f); // Make the last scroll look nearly instant @@ -405,7 +466,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { // The last batch had 100 documents action.startNextScroll(100); - // So the next request is going to have to wait an extra 100 seconds or so (base was 10, so 110ish) + // So the next request is going to have to wait an extra 100 seconds or so (base was 10 seconds, so 110ish) assertThat(client.lastScroll.get().request.scroll().keepAlive().seconds(), either(equalTo(110L)).or(equalTo(109L))); // Now we can simulate a response and check the delay that we used for the task @@ -422,10 +483,14 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { assertEquals(capturedDelay.get(), testTask.getStatus().getThrottled()); } - private long retryTestCase(boolean failWithRejection) throws Exception { + /** + * Execute a bulk retry test case. The total number of failures is random and the number of retries attempted is set to + * testRequest.getMaxRetries and controled by the failWithRejection parameter. + */ + private void bulkRetryTestCase(boolean failWithRejection) throws Exception { int totalFailures = randomIntBetween(1, testRequest.getMaxRetries()); int size = randomIntBetween(1, 100); - int retryAttempts = totalFailures - (failWithRejection ? 1 : 0); + testRequest.setMaxRetries(totalFailures - (failWithRejection ? 1 : 0)); client.bulksToReject = client.bulksAttempts.get() + totalFailures; /* @@ -433,13 +498,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { * deal with it. We just wait for it to happen. */ CountDownLatch successLatch = new CountDownLatch(1); - DummyAbstractAsyncBulkByScrollAction action = new DummyAbstractAsyncBulkByScrollAction() { - @Override - BackoffPolicy backoffPolicy() { - // Force a backoff time of 0 to prevent sleeping - return constantBackoff(timeValueMillis(0), retryAttempts); - } - + DummyAbstractAsyncBulkByScrollAction action = new DummyActionWithoutBackoff() { @Override void startNextScroll(int lastBatchSize) { successLatch.countDown(); @@ -459,14 +518,13 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { } else { successLatch.await(10, TimeUnit.SECONDS); } - return retryAttempts; } /** * The default retry time matches what we say it is in the javadoc for the request. */ public void testDefaultRetryTimes() { - Iterator policy = new DummyAbstractAsyncBulkByScrollAction().backoffPolicy().iterator(); + Iterator policy = new DummyAbstractAsyncBulkByScrollAction().buildBackoffPolicy().iterator(); long millis = 0; while (policy.hasNext()) { millis += policy.next().millis(); @@ -625,7 +683,22 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { } } + /** + * An extension to {@linkplain DummyAbstractAsyncBulkByScrollAction} that uses a 0 delaying backoff policy. + */ + private class DummyActionWithoutBackoff extends DummyAbstractAsyncBulkByScrollAction { + @Override + BackoffPolicy buildBackoffPolicy() { + // Force a backoff time of 0 to prevent sleeping + return constantBackoff(timeValueMillis(0), testRequest.getMaxRetries()); + } + } + private static class DummyAbstractBulkByScrollRequest extends AbstractBulkByScrollRequest { + public DummyAbstractBulkByScrollRequest(SearchRequest searchRequest) { + super(searchRequest); + } + @Override protected DummyAbstractBulkByScrollRequest self() { return this; @@ -635,11 +708,23 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { private class MyMockClient extends FilterClient { private final List scrollsCleared = new ArrayList<>(); private final AtomicInteger bulksAttempts = new AtomicInteger(); + private final AtomicInteger searchAttempts = new AtomicInteger(); + private final AtomicInteger scrollAttempts = new AtomicInteger(); private final AtomicReference> lastHeaders = new AtomicReference<>(); private final AtomicReference lastRefreshRequest = new AtomicReference<>(); + /** + * Last search attempt that wasn't rejected outright. + */ + private final AtomicReference> lastSearch = new AtomicReference<>(); + /** + * Last scroll attempt that wasn't rejected outright. + */ private final AtomicReference> lastScroll = new AtomicReference<>(); + private int bulksToReject = 0; + private int searchesToReject = 0; + private int scrollsToReject = 0; public MyMockClient(Client in) { super(in); @@ -661,7 +746,19 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { listener.onResponse(null); return; } + if (request instanceof SearchRequest) { + if (searchAttempts.incrementAndGet() <= searchesToReject) { + listener.onFailure(wrappedRejectedException()); + return; + } + lastSearch.set(new RequestAndListener<>((SearchRequest) request, (ActionListener) listener)); + return; + } if (request instanceof SearchScrollRequest) { + if (scrollAttempts.incrementAndGet() <= scrollsToReject) { + listener.onFailure(wrappedRejectedException()); + return; + } lastScroll.set(new RequestAndListener<>((SearchScrollRequest) request, (ActionListener) listener)); return; } @@ -715,6 +812,25 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { } super.doExecute(action, request, listener); } + + private Throwable wrappedRejectedException() { + Exception e = new EsRejectedExecutionException(); + int wraps = randomIntBetween(0, 4); + for (int i = 0; i < wraps; i++) { + switch (randomIntBetween(0, 2)) { + case 0: + e = new SearchPhaseExecutionException("test", "test failure", e, new ShardSearchFailure[0]); + continue; + case 1: + e = new ReduceSearchPhaseException("test", "test failure", e, new ShardSearchFailure[0]); + continue; + case 2: + e = new ElasticsearchException(e); + continue; + } + } + return e; + } } private static class RequestAndListener, Response> { diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkByScrollTaskTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkByScrollTaskTests.java index ee07c4a9278..d64c69ba362 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkByScrollTaskTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkByScrollTaskTests.java @@ -127,25 +127,28 @@ public class BulkByScrollTaskTests extends ESTestCase { } public void testStatusHatesNegatives() { - expectThrows(IllegalArgumentException.class, status(-1, 0, 0, 0, 0, 0, 0, 0)); - expectThrows(IllegalArgumentException.class, status(0, -1, 0, 0, 0, 0, 0, 0)); - expectThrows(IllegalArgumentException.class, status(0, 0, -1, 0, 0, 0, 0, 0)); - expectThrows(IllegalArgumentException.class, status(0, 0, 0, -1, 0, 0, 0, 0)); - expectThrows(IllegalArgumentException.class, status(0, 0, 0, 0, -1, 0, 0, 0)); - expectThrows(IllegalArgumentException.class, status(0, 0, 0, 0, 0, -1, 0, 0)); - expectThrows(IllegalArgumentException.class, status(0, 0, 0, 0, 0, 0, -1, 0)); - expectThrows(IllegalArgumentException.class, status(0, 0, 0, 0, 0, 0, 0, -1)); + checkStatusNegatives(-1, 0, 0, 0, 0, 0, 0, 0, 0, "total"); + checkStatusNegatives(0, -1, 0, 0, 0, 0, 0, 0, 0, "updated"); + checkStatusNegatives(0, 0, -1, 0, 0, 0, 0, 0, 0, "created"); + checkStatusNegatives(0, 0, 0, -1, 0, 0, 0, 0, 0, "deleted"); + checkStatusNegatives(0, 0, 0, 0, -1, 0, 0, 0, 0, "batches"); + checkStatusNegatives(0, 0, 0, 0, 0, -1, 0, 0, 0, "versionConflicts"); + checkStatusNegatives(0, 0, 0, 0, 0, 0, -1, 0, 0, "noops"); + checkStatusNegatives(0, 0, 0, 0, 0, 0, 0, -1, 0, "bulkRetries"); + checkStatusNegatives(0, 0, 0, 0, 0, 0, 0, 0, -1, "searchRetries"); } /** * Build a task status with only some values. Used for testing negative values. */ - private ThrowingRunnable status(long total, long updated, long created, long deleted, int batches, long versionConflicts, - long noops, long retries) { + private void checkStatusNegatives(long total, long updated, long created, long deleted, int batches, long versionConflicts, + long noops, long bulkRetries, long searchRetries, String fieldName) { TimeValue throttle = parseTimeValue(randomPositiveTimeValue(), "test"); TimeValue throttledUntil = parseTimeValue(randomPositiveTimeValue(), "test"); - return () -> new BulkByScrollTask.Status(-1, 0, 0, 0, 0, 0, 0, 0, throttle, 0f, null, throttledUntil); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(total, updated, created, + deleted, batches, versionConflicts, noops, bulkRetries, searchRetries, throttle, 0f, null, throttledUntil)); + assertEquals(e.getMessage(), fieldName + " must be greater than 0 but was [-1]"); } /** diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkIndexByScrollResponseMatcher.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkIndexByScrollResponseMatcher.java index b34dfc4a595..5bce3260929 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkIndexByScrollResponseMatcher.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkIndexByScrollResponseMatcher.java @@ -118,7 +118,7 @@ public class BulkIndexByScrollResponseMatcher extends TypeSafeMatcher, - Response extends BulkIndexByScrollResponse, - Builder extends AbstractBulkIndexByScrollRequestBuilder> - Response testCancel(ESIntegTestCase test, Builder request, String actionToCancel) throws Exception { + Builder extends AbstractBulkIndexByScrollRequestBuilder> + BulkIndexByScrollResponse testCancel(ESIntegTestCase test, Builder request, String actionToCancel) throws Exception { test.indexRandom(true, client().prepareIndex("source", "test", "1").setSource("foo", "a"), client().prepareIndex("source", "test", "2").setSource("foo", "a")); request.source("source").script(new Script("sticky", ScriptType.INLINE, "native", emptyMap())); request.source().setSize(1); - ListenableActionFuture response = request.execute(); + ListenableActionFuture response = request.execute(); // Wait until the script is on the first document. barrier.await(30, TimeUnit.SECONDS); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java index 5a0196159de..dfa17ae4778 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java @@ -43,7 +43,7 @@ public class RethrottleTests extends ReindexTestCase { testCase(updateByQuery().source("test"), UpdateByQueryAction.NAME); } - private void testCase(AbstractBulkIndexByScrollRequestBuilder request, String actionName) + private void testCase(AbstractBulkIndexByScrollRequestBuilder request, String actionName) throws Exception { // Use a single shard so the reindex has to happen in multiple batches client().admin().indices().prepareCreate("test").setSettings("index.number_of_shards", 1).get(); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java new file mode 100644 index 00000000000..0b557898552 --- /dev/null +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java @@ -0,0 +1,155 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.reindex; + +import org.elasticsearch.action.ListenableActionFuture; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.bulk.Retry; +import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.MockSearchService; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.function.IntFunction; + +import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.greaterThan; + +/** + * Integration test for retry behavior. Useful because retrying relies on the way that the rest of Elasticsearch throws exceptions and unit + * tests won't verify that. + */ +public class RetryTests extends ReindexTestCase { + /** + * The number of concurrent requests to test. + */ + private static final int CONCURRENT = 12; + /** + * Enough docs that the requests will likely step on each other. + */ + private static final int DOC_COUNT = 200; + + /** + * Lower the queue sizes to be small enough that both bulk and searches will time out and have to be retried. + */ + @Override + protected Settings nodeSettings(int nodeOrdinal) { + Settings.Builder settings = Settings.builder().put(super.nodeSettings(nodeOrdinal)); + settings.put("threadpool.bulk.queue_size", 1); + settings.put("threadpool.bulk.size", 1); + settings.put("threadpool.search.queue_size", 1); + settings.put("threadpool.search.size", 1); + return settings.build(); + } + + /** + * Disable search context leak detection because we expect leaks when there is an {@link EsRejectedExecutionException} queueing the + * reduce phase. + */ + @Override + protected Collection> getMockPlugins() { + List> mockPlugins = new ArrayList<>(); + for (Class plugin: super.getMockPlugins()) { + if (plugin.equals(MockSearchService.TestPlugin.class)) { + continue; + } + mockPlugins.add(plugin); + } + return mockPlugins; + } + + public void testReindex() throws Exception { + setupSourceIndex("source"); + testCase(true, i -> reindex().source("source").destination("dest" + i)); + } + + public void testUpdateByQuery() throws Exception { + for (int i = 0; i < CONCURRENT; i++) { + setupSourceIndex("source" + i); + } + testCase(false, i -> updateByQuery().source("source" + i)); + } + + private void testCase(boolean expectCreated, IntFunction> requestBuilder) + throws Exception { + List> futures = new ArrayList<>(CONCURRENT); + for (int i = 0; i < CONCURRENT; i++) { + AbstractBulkIndexByScrollRequestBuilder request = requestBuilder.apply(i); + // Make sure we use more than one batch so we get the full reindex behavior + request.source().setSize(DOC_COUNT / randomIntBetween(2, 10)); + // Use a low, random initial wait so we are unlikely collide with others retrying. + request.setRetryBackoffInitialTime(timeValueMillis(randomIntBetween(10, 300))); + futures.add(request.execute()); + } + + // Finish all the requests + List responses = new ArrayList<>(CONCURRENT); + for (ListenableActionFuture future : futures) { + responses.add(future.get()); + } + + // Now check them + long bulkRetries = 0; + long searchRetries = 0; + BulkIndexByScrollResponseMatcher matcher = matcher(); + if (expectCreated) { + matcher.created(DOC_COUNT); + } else { + matcher.updated(DOC_COUNT); + } + for (BulkIndexByScrollResponse response : responses) { + assertThat(response, matcher); + bulkRetries += response.getBulkRetries(); + searchRetries += response.getSearchRetries(); + } + + // We expect at least one retry or this test isn't very useful + assertThat(bulkRetries, greaterThan(0L)); + assertThat(searchRetries, greaterThan(0L)); + } + + private void setupSourceIndex(String name) { + try { + // Build the test index with a single shard so we can be sure that a search request *can* complete with the one thread + assertAcked(client().admin().indices().prepareCreate(name).setSettings( + "index.number_of_shards", 1, + "index.number_of_replicas", 0).get()); + waitForRelocation(ClusterHealthStatus.GREEN); + // Build the test data. Don't use indexRandom because that won't work consistently with such small thread pools. + BulkRequestBuilder bulk = client().prepareBulk(); + for (int i = 0; i < DOC_COUNT; i++) { + bulk.add(client().prepareIndex(name, "test").setSource("foo", "bar " + i)); + } + Retry retry = Retry.on(EsRejectedExecutionException.class).policy(BackoffPolicy.exponentialBackoff()); + BulkResponse response = retry.withSyncBackoff(client(), bulk.request()); + assertFalse(response.buildFailureMessage(), response.hasFailures()); + refresh(name); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java index 55a665cfa7e..d1cb77361bb 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java @@ -138,7 +138,7 @@ public class RoundTripTests extends ESTestCase { private BulkByScrollTask.Status randomStatus() { return new BulkByScrollTask.Status(randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), - randomInt(Integer.MAX_VALUE), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), + randomInt(Integer.MAX_VALUE), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), parseTimeValue(randomPositiveTimeValue(), "test"), abs(random().nextFloat()), random().nextBoolean() ? null : randomSimpleString(random()), parseTimeValue(randomPositiveTimeValue(), "test")); } @@ -210,7 +210,8 @@ public class RoundTripTests extends ESTestCase { assertEquals(expected.getBatches(), actual.getBatches()); assertEquals(expected.getVersionConflicts(), actual.getVersionConflicts()); assertEquals(expected.getNoops(), actual.getNoops()); - assertEquals(expected.getRetries(), actual.getRetries()); + assertEquals(expected.getBulkRetries(), actual.getBulkRetries()); + assertEquals(expected.getSearchRetries(), actual.getSearchRetries()); assertEquals(expected.getThrottled(), actual.getThrottled()); assertEquals(expected.getRequestsPerSecond(), actual.getRequestsPerSecond(), 0f); assertEquals(expected.getReasonCancelled(), actual.getReasonCancelled()); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryRequestTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryRequestTests.java index f6780729143..5d17e1d88fd 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryRequestTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryRequestTests.java @@ -19,12 +19,12 @@ package org.elasticsearch.index.reindex; -import java.util.List; - import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.test.ESTestCase; +import java.util.List; + import static org.apache.lucene.util.TestUtil.randomSimpleString; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.hasSize; diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/100_sort.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/100_sort.yaml new file mode 100644 index 00000000000..096e57fa4e1 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/100_sort.yaml @@ -0,0 +1,35 @@ +--- +"Test sort Processor": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "sort" : { + "field" : "values" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + type: test + id: 1 + pipeline: "my_pipeline" + body: > + { + "values": ["foo", "bar", "baz"] + } + + - do: + get: + index: test + type: test + id: 1 + - match: { _source.values: ["bar", "baz", "foo"] }