From 76de819c75d3712ef0a9df2196c3b3b30a93304b Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Tue, 14 Mar 2017 14:37:47 +0700 Subject: [PATCH 01/20] SOLR-9835: Create another replication mode for SolrCloud --- .../solr/hadoop/TreeMergeOutputFormat.java | 3 +- .../apache/solr/cloud/ElectionContext.java | 20 +- .../OverseerCollectionMessageHandler.java | 1 + .../apache/solr/cloud/RecoveryStrategy.java | 29 +- .../solr/cloud/ReplicateFromLeader.java | 124 +++++ .../org/apache/solr/cloud/ZkController.java | 31 +- .../org/apache/solr/core/CoreContainer.java | 7 + .../org/apache/solr/handler/IndexFetcher.java | 34 +- .../solr/handler/ReplicationHandler.java | 28 +- .../handler/admin/CollectionsHandler.java | 4 +- .../org/apache/solr/update/CommitTracker.java | 5 + .../solr/update/DirectUpdateHandler2.java | 31 +- .../solr/update/HdfsTransactionLog.java | 50 ++ .../apache/solr/update/SolrIndexSplitter.java | 3 +- .../apache/solr/update/SolrIndexWriter.java | 6 +- .../apache/solr/update/TransactionLog.java | 50 ++ .../org/apache/solr/update/UpdateCommand.java | 1 + .../org/apache/solr/update/UpdateLog.java | 171 ++++++- .../processor/DistributedUpdateProcessor.java | 38 +- .../org/apache/solr/util/TestInjection.java | 54 +++ .../conf/schema.xml | 31 ++ .../conf/solrconfig.xml | 48 ++ .../solr/cloud/BasicDistributedZk2Test.java | 6 + .../solr/cloud/BasicDistributedZkTest.java | 9 +- .../cloud/ChaosMonkeyNothingIsSafeTest.java | 7 + .../apache/solr/cloud/ForceLeaderTest.java | 6 + .../apache/solr/cloud/HttpPartitionTest.java | 7 + .../LeaderInitiatedRecoveryOnCommitTest.java | 7 + .../solr/cloud/OnlyLeaderIndexesTest.java | 435 ++++++++++++++++++ .../cloud/RecoveryAfterSoftCommitTest.java | 7 +- .../org/apache/solr/cloud/ShardSplitTest.java | 6 + .../apache/solr/cloud/TestCloudRecovery.java | 16 +- .../apache/solr/cloud/TestCollectionAPI.java | 6 +- .../hdfs/HdfsBasicDistributedZkTest.java | 7 +- .../update/TestInPlaceUpdatesDistrib.java | 23 +- .../solrj/request/CollectionAdminRequest.java | 6 + .../solr/common/cloud/DocCollection.java | 12 + .../solr/common/cloud/ZkStateReader.java | 1 + .../cloud/AbstractFullDistribZkTestBase.java | 14 +- 39 files changed, 1309 insertions(+), 35 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java create mode 100644 solr/core/src/test-files/solr/configsets/cloud-minimal-inplace-updates/conf/schema.xml create mode 100644 solr/core/src/test-files/solr/configsets/cloud-minimal-inplace-updates/conf/solrconfig.xml create mode 100644 solr/core/src/test/org/apache/solr/cloud/OnlyLeaderIndexesTest.java diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java index e3487adad21..cac57c35aae 100644 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java +++ b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java @@ -163,7 +163,8 @@ public class TreeMergeOutputFormat extends FileOutputFormat // Set Solr's commit data so the created index is usable by SolrCloud. E.g. Currently SolrCloud relies on // commitTimeMSec in the commit data to do replication. - SolrIndexWriter.setCommitData(writer); + //TODO no commitUpdateCommand + SolrIndexWriter.setCommitData(writer, -1); timer = new RTimer(); LOG.info("Optimizing Solr: Closing index writer"); diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java index d3ad3224382..223a5395201 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java +++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import org.apache.hadoop.fs.Path; @@ -420,7 +421,24 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase { try { // we must check LIR before registering as leader checkLIR(coreName, allReplicasInLine); - + + boolean onlyLeaderIndexes = zkController.getClusterState().getCollection(collection).getRealtimeReplicas() == 1; + if (onlyLeaderIndexes) { + // stop replicate from old leader + zkController.stopReplicationFromLeader(coreName); + if (weAreReplacement) { + try (SolrCore core = cc.getCore(coreName)) { + Future future = core.getUpdateHandler().getUpdateLog().recoverFromCurrentLog(); + if (future != null) { + log.info("Replaying tlog before become new leader"); + future.get(); + } else { + log.info("New leader does not have old tlog to replay"); + } + } + } + } + super.runLeaderProcess(weAreReplacement, 0); try (SolrCore core = cc.getCore(coreName)) { if (core != null) { diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java index 00eb12db692..4d64a0087e5 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java @@ -131,6 +131,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler ZkStateReader.REPLICATION_FACTOR, "1", ZkStateReader.MAX_SHARDS_PER_NODE, "1", ZkStateReader.AUTO_ADD_REPLICAS, "false", + ZkStateReader.REALTIME_REPLICAS, "-1", DocCollection.RULE, null, SNITCH, null)); diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java index 8865c08ddb1..cb6c69c8aa9 100644 --- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java +++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java @@ -118,7 +118,8 @@ public class RecoveryStrategy extends Thread implements Closeable { private boolean recoveringAfterStartup; private CoreContainer cc; private volatile HttpUriRequest prevSendPreRecoveryHttpUriRequest; - + private boolean onlyLeaderIndexes; + protected RecoveryStrategy(CoreContainer cc, CoreDescriptor cd, RecoveryListener recoveryListener) { this.cc = cc; this.coreName = cd.getName(); @@ -128,6 +129,8 @@ public class RecoveryStrategy extends Thread implements Closeable { zkStateReader = zkController.getZkStateReader(); baseUrl = zkController.getBaseUrl(); coreZkNodeName = cd.getCloudDescriptor().getCoreNodeName(); + String collection = cd.getCloudDescriptor().getCollectionName(); + onlyLeaderIndexes = zkStateReader.getClusterState().getCollection(collection).getRealtimeReplicas() == 1; } final public int getWaitForUpdatesWithStaleStatePauseMilliSeconds() { @@ -260,7 +263,7 @@ public class RecoveryStrategy extends Thread implements Closeable { UpdateRequest ureq = new UpdateRequest(); ureq.setParams(new ModifiableSolrParams()); ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true); - ureq.getParams().set(UpdateParams.OPEN_SEARCHER, false); + ureq.getParams().set(UpdateParams.OPEN_SEARCHER, onlyLeaderIndexes); ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true).process( client); } @@ -309,7 +312,8 @@ public class RecoveryStrategy extends Thread implements Closeable { return; } - boolean firstTime = true; + // we temporary ignore peersync for realtimeReplicas mode + boolean firstTime = !onlyLeaderIndexes; List recentVersions; try (UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates()) { @@ -361,6 +365,10 @@ public class RecoveryStrategy extends Thread implements Closeable { } } + if (onlyLeaderIndexes) { + zkController.stopReplicationFromLeader(coreName); + } + Future replayFuture = null; while (!successfulRecovery && !isInterrupted() && !isClosed()) { // don't use interruption or it will close channels though try { @@ -514,6 +522,9 @@ public class RecoveryStrategy extends Thread implements Closeable { if (successfulRecovery) { LOG.info("Registering as Active after recovery."); try { + if (onlyLeaderIndexes) { + zkController.startReplicationFromLeader(coreName); + } zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE); } catch (Exception e) { LOG.error("Could not publish as ACTIVE after succesful recovery", e); @@ -587,8 +598,20 @@ public class RecoveryStrategy extends Thread implements Closeable { LOG.info("Finished recovery process, successful=[{}]", Boolean.toString(successfulRecovery)); } + public static Runnable testing_beforeReplayBufferingUpdates; + final private Future replay(SolrCore core) throws InterruptedException, ExecutionException { + if (testing_beforeReplayBufferingUpdates != null) { + testing_beforeReplayBufferingUpdates.run(); + } + if (onlyLeaderIndexes) { + // roll over all updates during buffering to new tlog, make RTG available + SolrQueryRequest req = new LocalSolrQueryRequest(core, + new ModifiableSolrParams()); + core.getUpdateHandler().getUpdateLog().copyOverBufferingUpdates(new CommitUpdateCommand(req, false)); + return null; + } Future future = core.getUpdateHandler().getUpdateLog().applyBufferedUpdates(); if (future == null) { // no replay needed\ diff --git a/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java b/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java new file mode 100644 index 00000000000..d7fded907a9 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.solr.cloud; + +import java.lang.invoke.MethodHandles; + +import org.apache.lucene.index.IndexCommit; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.util.NamedList; +import org.apache.solr.core.CoreContainer; +import org.apache.solr.core.SolrConfig; +import org.apache.solr.core.SolrCore; +import org.apache.solr.handler.ReplicationHandler; +import org.apache.solr.request.LocalSolrQueryRequest; +import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.update.CommitUpdateCommand; +import org.apache.solr.update.SolrIndexWriter; +import org.apache.solr.update.UpdateLog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ReplicateFromLeader { + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private CoreContainer cc; + private String coreName; + + private ReplicationHandler replicationProcess; + private long lastVersion = 0; + + public ReplicateFromLeader(CoreContainer cc, String coreName) { + this.cc = cc; + this.coreName = coreName; + } + + public void startReplication() throws InterruptedException { + try (SolrCore core = cc.getCore(coreName)) { + if (core == null) { + if (cc.isShutDown()) { + return; + } else { + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "SolrCore not found:" + coreName + " in " + cc.getCoreNames()); + } + } + SolrConfig.UpdateHandlerInfo uinfo = core.getSolrConfig().getUpdateHandlerInfo(); + String pollIntervalStr = "00:00:03"; + if (uinfo.autoCommmitMaxTime != -1) { + pollIntervalStr = toPollIntervalStr(uinfo.autoCommmitMaxTime/2); + } else if (uinfo.autoSoftCommmitMaxTime != -1) { + pollIntervalStr = toPollIntervalStr(uinfo.autoSoftCommmitMaxTime/2); + } + + NamedList slaveConfig = new NamedList(); + slaveConfig.add("fetchFromLeader", true); + slaveConfig.add("pollInterval", pollIntervalStr); + NamedList replicationConfig = new NamedList(); + replicationConfig.add("slave", slaveConfig); + + String lastCommitVersion = getCommitVersion(core); + if (lastCommitVersion != null) { + lastVersion = Long.parseLong(lastCommitVersion); + } + + replicationProcess = new ReplicationHandler(); + replicationProcess.setPollListener((solrCore, pollSuccess) -> { + if (pollSuccess) { + String commitVersion = getCommitVersion(core); + if (commitVersion == null) return; + if (Long.parseLong(commitVersion) == lastVersion) return; + UpdateLog updateLog = solrCore.getUpdateHandler().getUpdateLog(); + SolrQueryRequest req = new LocalSolrQueryRequest(core, + new ModifiableSolrParams()); + CommitUpdateCommand cuc = new CommitUpdateCommand(req, false); + cuc.setVersion(Long.parseLong(commitVersion)); + updateLog.copyOverOldUpdates(cuc); + lastVersion = Long.parseLong(commitVersion); + } + }); + replicationProcess.init(replicationConfig); + replicationProcess.inform(core); + } + } + + public static String getCommitVersion(SolrCore solrCore) { + IndexCommit commit = solrCore.getDeletionPolicy().getLatestCommit(); + try { + String commitVersion = commit.getUserData().get(SolrIndexWriter.COMMIT_COMMAND_VERSION); + if (commitVersion == null) return null; + else return commitVersion; + } catch (Exception e) { + LOG.warn("Cannot get commit command version from index commit point ",e); + return null; + } + } + + private static String toPollIntervalStr(int ms) { + int sec = ms/1000; + int hour = sec / 3600; + sec = sec % 3600; + int min = sec / 60; + sec = sec % 60; + return hour + ":" + min + ":" + sec; + } + + public void stopReplication() { + replicationProcess.close(); + } +} diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java index 333acd419d7..a19b35111b1 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -189,6 +189,7 @@ public class ZkController { private LeaderElector overseerElector; + private Map replicateFromLeaders = new ConcurrentHashMap<>(); // for now, this can be null in tests, in which case recovery will be inactive, and other features // may accept defaults or use mocks rather than pulling things from a CoreContainer @@ -877,7 +878,7 @@ public class ZkController { coreName, baseUrl, cloudDesc.getCollectionName(), shardId); ZkNodeProps leaderProps = new ZkNodeProps(props); - + try { // If we're a preferred leader, insert ourselves at the head of the queue boolean joinAtHead = false; @@ -913,9 +914,16 @@ public class ZkController { // leader election perhaps? UpdateLog ulog = core.getUpdateHandler().getUpdateLog(); - + boolean onlyLeaderIndexes = zkStateReader.getClusterState().getCollection(collection).getRealtimeReplicas() == 1; + boolean isReplicaInOnlyLeaderIndexes = onlyLeaderIndexes && !isLeader; + if (isReplicaInOnlyLeaderIndexes) { + String commitVersion = ReplicateFromLeader.getCommitVersion(core); + if (commitVersion != null) { + ulog.copyOverOldUpdates(Long.parseLong(commitVersion)); + } + } // we will call register again after zk expiration and on reload - if (!afterExpiration && !core.isReloaded() && ulog != null) { + if (!afterExpiration && !core.isReloaded() && ulog != null && !isReplicaInOnlyLeaderIndexes) { // disable recovery in case shard is in construction state (for shard splits) Slice slice = getClusterState().getSlice(collection, shardId); if (slice.getState() != Slice.State.CONSTRUCTION || !isLeader) { @@ -934,6 +942,9 @@ public class ZkController { boolean didRecovery = checkRecovery(recoverReloadedCores, isLeader, skipRecovery, collection, coreZkNodeName, core, cc, afterExpiration); if (!didRecovery) { + if (isReplicaInOnlyLeaderIndexes) { + startReplicationFromLeader(coreName); + } publish(desc, Replica.State.ACTIVE); } @@ -948,6 +959,20 @@ public class ZkController { } } + public void startReplicationFromLeader(String coreName) throws InterruptedException { + ReplicateFromLeader replicateFromLeader = new ReplicateFromLeader(cc, coreName); + if (replicateFromLeaders.putIfAbsent(coreName, replicateFromLeader) == null) { + replicateFromLeader.startReplication(); + } + } + + public void stopReplicationFromLeader(String coreName) { + ReplicateFromLeader replicateFromLeader = replicateFromLeaders.remove(coreName); + if (replicateFromLeader != null) { + replicateFromLeader.stopReplication(); + } + } + // timeoutms is the timeout for the first call to get the leader - there is then // a longer wait to make sure that leader matches our local state private String getLeader(final CloudDescriptor cloudDesc, int timeoutms) { diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java index b9597ae985f..0de671e839d 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -1137,6 +1137,13 @@ public class CoreContainer { log.info("Reloading SolrCore '{}' using configuration from {}", cd.getName(), coreConfig.getName()); SolrCore newCore = core.reload(coreConfig); registerCore(cd.getName(), newCore, false, false); + if (getZkController() != null) { + boolean onlyLeaderIndexes = getZkController().getClusterState().getCollection(cd.getCollectionName()).getRealtimeReplicas() == 1; + if (onlyLeaderIndexes && !cd.getCloudDescriptor().isLeader()) { + getZkController().stopReplicationFromLeader(core.getName()); + getZkController().startReplicationFromLeader(newCore.getName()); + } + } } catch (SolrCoreState.CoreIsClosedException e) { throw e; } catch (Exception e) { diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java index 8634aeeb002..a07496fcdf1 100644 --- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java +++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java @@ -68,8 +68,11 @@ import org.apache.solr.client.solrj.impl.HttpClientUtil; import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder; import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.cloud.CloudDescriptor; +import org.apache.solr.cloud.ZkController; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; +import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.ExecutorUtil; @@ -115,7 +118,7 @@ public class IndexFetcher { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private final String masterUrl; + private String masterUrl; final ReplicationHandler replicationHandler; @@ -150,6 +153,8 @@ public class IndexFetcher { private boolean useExternalCompression = false; + private boolean fetchFromLeader = false; + private final HttpClient myHttpClient; private Integer connTimeout; @@ -167,11 +172,15 @@ public class IndexFetcher { public IndexFetcher(final NamedList initArgs, final ReplicationHandler handler, final SolrCore sc) { solrCore = sc; + Object fetchFromLeader = initArgs.get(FETCH_FROM_LEADER); + if (fetchFromLeader != null && fetchFromLeader instanceof Boolean) { + this.fetchFromLeader = (boolean) fetchFromLeader; + } String masterUrl = (String) initArgs.get(MASTER_URL); - if (masterUrl == null) + if (masterUrl == null && !this.fetchFromLeader) throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "'masterUrl' is required for a slave"); - if (masterUrl.endsWith(ReplicationHandler.PATH)) { + if (masterUrl != null && masterUrl.endsWith(ReplicationHandler.PATH)) { masterUrl = masterUrl.substring(0, masterUrl.length()-12); LOG.warn("'masterUrl' must be specified without the "+ReplicationHandler.PATH+" suffix"); } @@ -298,6 +307,15 @@ public class IndexFetcher { } try { + if (fetchFromLeader) { + Replica replica = getLeaderReplica(); + CloudDescriptor cd = solrCore.getCoreDescriptor().getCloudDescriptor(); + if (cd.getCoreNodeName().equals(replica.getName())) { + return false; + } + masterUrl = replica.getCoreUrl(); + LOG.info("Updated masterUrl to " + masterUrl); + } //get the current 'replicateable' index version in the master NamedList response; try { @@ -404,7 +422,7 @@ public class IndexFetcher { isFullCopyNeeded = true; } - if (!isFullCopyNeeded) { + if (!isFullCopyNeeded && !fetchFromLeader) { // a searcher might be using some flushed but not committed segments // because of soft commits (which open a searcher on IW's data) // so we need to close the existing searcher on the last commit @@ -565,6 +583,14 @@ public class IndexFetcher { } } + private Replica getLeaderReplica() throws InterruptedException { + ZkController zkController = solrCore.getCoreDescriptor().getCoreContainer().getZkController(); + CloudDescriptor cd = solrCore.getCoreDescriptor().getCloudDescriptor(); + Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry( + cd.getCollectionName(), cd.getShardId()); + return leaderReplica; + } + private void cleanup(final SolrCore core, Directory tmpIndexDir, Directory indexDir, boolean deleteTmpIdxDir, File tmpTlogDir, boolean successfulInstall) throws IOException { try { diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java index cdbadc4eeca..e40b2c38311 100644 --- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java @@ -209,6 +209,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw private Long pollIntervalNs; private String pollIntervalStr; + private PollListener pollListener; + public interface PollListener { + void onComplete(SolrCore solrCore, boolean pollSuccess) throws IOException; + } + /** * Disable the timer task for polling */ @@ -218,6 +223,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw return pollIntervalStr; } + public void setPollListener(PollListener pollListener) { + this.pollListener = pollListener; + } + @Override public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception { rsp.setHttpCaching(false); @@ -1142,7 +1151,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw try { LOG.debug("Polling for index modifications"); markScheduledExecutionStart(); - doFetch(null, false); + boolean pollSuccess = doFetch(null, false); + if (pollListener != null) pollListener.onComplete(core, pollSuccess); } catch (Exception e) { LOG.error("Exception in fetching index", e); } @@ -1328,6 +1338,20 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw }); } + public void close() { + if (executorService != null) executorService.shutdown(); + if (pollingIndexFetcher != null) { + pollingIndexFetcher.destroy(); + } + if (currentIndexFetcher != null && currentIndexFetcher != pollingIndexFetcher) { + currentIndexFetcher.destroy(); + } + ExecutorUtil.shutdownAndAwaitTermination(restoreExecutor); + if (restoreFuture != null) { + restoreFuture.cancel(false); + } + } + /** * Register a listener for postcommit/optimize * @@ -1680,6 +1704,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw public static final String MASTER_URL = "masterUrl"; + public static final String FETCH_FROM_LEADER = "fetchFromLeader"; + public static final String STATUS = "status"; public static final String COMMAND = "command"; diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java index d7759cadde0..2e17af6df23 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java @@ -115,6 +115,7 @@ import static org.apache.solr.common.cloud.DocCollection.STATE_FORMAT; import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS; import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP; import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE; +import static org.apache.solr.common.cloud.ZkStateReader.REALTIME_REPLICAS; import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP; import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP; import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR; @@ -404,7 +405,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission STATE_FORMAT, AUTO_ADD_REPLICAS, RULE, - SNITCH); + SNITCH, + REALTIME_REPLICAS); if (props.get(STATE_FORMAT) == null) { props.put(STATE_FORMAT, "2"); diff --git a/solr/core/src/java/org/apache/solr/update/CommitTracker.java b/solr/core/src/java/org/apache/solr/update/CommitTracker.java index 61f0c352630..9c09ebeb3fd 100644 --- a/solr/core/src/java/org/apache/solr/update/CommitTracker.java +++ b/solr/core/src/java/org/apache/solr/update/CommitTracker.java @@ -207,6 +207,11 @@ public final class CommitTracker implements Runnable { command.openSearcher = openSearcher; command.waitSearcher = waitSearcher; command.softCommit = softCommit; + if (core.getCoreDescriptor().getCloudDescriptor() != null + && core.getCoreDescriptor().getCloudDescriptor().isLeader() + && !softCommit) { + command.version = core.getUpdateHandler().getUpdateLog().getVersionInfo().getNewClock(); + } // no need for command.maxOptimizeSegments = 1; since it is not optimizing // we increment this *before* calling commit because it was causing a race diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java index 4592bcf980b..abb5512c841 100644 --- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java +++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java @@ -45,7 +45,9 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.util.BytesRefHash; +import org.apache.solr.cloud.ZkController; import org.apache.solr.common.SolrException; +import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; @@ -123,6 +125,14 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState commitWithinSoftCommit = updateHandlerInfo.commitWithinSoftCommit; indexWriterCloseWaitsForMerges = updateHandlerInfo.indexWriterCloseWaitsForMerges; + ZkController zkController = core.getCoreDescriptor().getCoreContainer().getZkController(); + if (zkController != null) { + DocCollection dc = zkController.getClusterState().getCollection(core.getCoreDescriptor().getCollectionName()); + if (dc.getRealtimeReplicas() == 1) { + commitWithinSoftCommit = false; + commitTracker.setOpenSearcher(true); + } + } } @@ -233,6 +243,11 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState cmd.overwrite = false; } try { + if ( (cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0) { + if (ulog != null) ulog.add(cmd); + return 1; + } + if (cmd.overwrite) { // Check for delete by query commands newer (i.e. reordered). This // should always be null on a leader @@ -404,6 +419,11 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState deleteByIdCommands.increment(); deleteByIdCommandsCumulative.mark(); + if ( (cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0 ) { + if (ulog != null) ulog.delete(cmd); + return; + } + Term deleteTerm = new Term(idField.getName(), cmd.getIndexedId()); // SolrCore.verbose("deleteDocuments",deleteTerm,writer); RefCounted iw = solrCoreState.getIndexWriter(core); @@ -463,6 +483,11 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState deleteByQueryCommandsCumulative.mark(); boolean madeIt=false; try { + if ( (cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0) { + if (ulog != null) ulog.deleteByQuery(cmd); + madeIt = true; + return; + } Query q = getQuery(cmd); boolean delAll = MatchAllDocsQuery.class == q.getClass(); @@ -563,7 +588,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState log.info("start "+cmd); RefCounted iw = solrCoreState.getIndexWriter(core); try { - SolrIndexWriter.setCommitData(iw.get()); + SolrIndexWriter.setCommitData(iw.get(), cmd.getVersion()); iw.get().prepareCommit(); } finally { iw.decref(); @@ -647,7 +672,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState // SolrCore.verbose("writer.commit() start writer=",writer); if (writer.hasUncommittedChanges()) { - SolrIndexWriter.setCommitData(writer); + SolrIndexWriter.setCommitData(writer, cmd.getVersion()); writer.commit(); } else { log.info("No uncommitted changes. Skipping IW.commit."); @@ -838,7 +863,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState } // todo: refactor this shared code (or figure out why a real CommitUpdateCommand can't be used) - SolrIndexWriter.setCommitData(writer); + SolrIndexWriter.setCommitData(writer, cmd.getVersion()); writer.commit(); synchronized (solrCoreState.getUpdateLock()) { diff --git a/solr/core/src/java/org/apache/solr/update/HdfsTransactionLog.java b/solr/core/src/java/org/apache/solr/update/HdfsTransactionLog.java index 90f6856b1cb..c478935d1ec 100644 --- a/solr/core/src/java/org/apache/solr/update/HdfsTransactionLog.java +++ b/solr/core/src/java/org/apache/solr/update/HdfsTransactionLog.java @@ -20,8 +20,10 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.Collection; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.TreeMap; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -368,6 +370,10 @@ public class HdfsTransactionLog extends TransactionLog { return new HDFSLogReader(startingPos); } + public LogReader getSortedReader(long startingPos) { + return new HDFSSortedLogReader(startingPos); + } + /** Returns a single threaded reverse reader */ @Override public ReverseReader getReverseReader() throws IOException { @@ -477,6 +483,50 @@ public class HdfsTransactionLog extends TransactionLog { } + public class HDFSSortedLogReader extends HDFSLogReader{ + private long startingPos; + private boolean inOrder = true; + private TreeMap versionToPos; + Iterator iterator; + + public HDFSSortedLogReader(long startingPos) { + super(startingPos); + this.startingPos = startingPos; + } + + @Override + public Object next() throws IOException, InterruptedException { + if (versionToPos == null) { + versionToPos = new TreeMap<>(); + Object o; + long pos = startingPos; + + long lastVersion = Long.MIN_VALUE; + while ( (o = super.next()) != null) { + List entry = (List) o; + long version = (Long) entry.get(UpdateLog.VERSION_IDX); + version = Math.abs(version); + versionToPos.put(version, pos); + pos = currentPos(); + + if (version < lastVersion) inOrder = false; + lastVersion = version; + } + fis.seek(startingPos); + } + + if (inOrder) { + return super.next(); + } else { + if (iterator == null) iterator = versionToPos.values().iterator(); + if (!iterator.hasNext()) return null; + long pos = iterator.next(); + if (pos != currentPos()) fis.seek(pos); + return super.next(); + } + } + } + public class HDFSReverseReader extends ReverseReader { FSDataFastInputStream fis; private LogCodec codec = new LogCodec(resolver) { diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java b/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java index a147b0fc357..e9950f2b42c 100644 --- a/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java +++ b/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java @@ -137,7 +137,8 @@ public class SolrIndexSplitter { // we commit explicitly instead of sending a CommitUpdateCommand through the processor chain // because the sub-shard cores will just ignore such a commit because the update log is not // in active state at this time. - SolrIndexWriter.setCommitData(iw); + //TODO no commitUpdateCommand + SolrIndexWriter.setCommitData(iw, -1); iw.commit(); success = true; } finally { diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java b/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java index 3c0c1a5a0cc..6a264f851dd 100644 --- a/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java +++ b/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java @@ -61,6 +61,7 @@ public class SolrIndexWriter extends IndexWriter { /** Stored into each Lucene commit to record the * System.currentTimeMillis() when commit was called. */ public static final String COMMIT_TIME_MSEC_KEY = "commitTimeMSec"; + public static final String COMMIT_COMMAND_VERSION = "commitCommandVer"; private final Object CLOSE_LOCK = new Object(); @@ -183,10 +184,11 @@ public class SolrIndexWriter extends IndexWriter { @SuppressForbidden(reason = "Need currentTimeMillis, commit time should be used only for debugging purposes, " + " but currently suspiciously used for replication as well") - public static void setCommitData(IndexWriter iw) { - log.info("Calling setCommitData with IW:" + iw.toString()); + public static void setCommitData(IndexWriter iw, long commitCommandVersion) { + log.info("Calling setCommitData with IW:" + iw.toString() + " commitCommandVersion:"+commitCommandVersion); final Map commitData = new HashMap<>(); commitData.put(COMMIT_TIME_MSEC_KEY, String.valueOf(System.currentTimeMillis())); + commitData.put(COMMIT_COMMAND_VERSION, String.valueOf(commitCommandVersion)); iw.setLiveCommitData(commitData.entrySet()); } diff --git a/solr/core/src/java/org/apache/solr/update/TransactionLog.java b/solr/core/src/java/org/apache/solr/update/TransactionLog.java index 5037b45c5ef..73328cf8536 100644 --- a/solr/core/src/java/org/apache/solr/update/TransactionLog.java +++ b/solr/core/src/java/org/apache/solr/update/TransactionLog.java @@ -29,9 +29,11 @@ import java.nio.file.Files; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.util.BytesRef; @@ -632,6 +634,10 @@ public class TransactionLog implements Closeable { return new LogReader(startingPos); } + public LogReader getSortedReader(long startingPos) { + return new SortedLogReader(startingPos); + } + /** Returns a single threaded reverse reader */ public ReverseReader getReverseReader() throws IOException { return new FSReverseReader(); @@ -715,6 +721,50 @@ public class TransactionLog implements Closeable { } + public class SortedLogReader extends LogReader { + private long startingPos; + private boolean inOrder = true; + private TreeMap versionToPos; + Iterator iterator; + + public SortedLogReader(long startingPos) { + super(startingPos); + this.startingPos = startingPos; + } + + @Override + public Object next() throws IOException, InterruptedException { + if (versionToPos == null) { + versionToPos = new TreeMap<>(); + Object o; + long pos = startingPos; + + long lastVersion = Long.MIN_VALUE; + while ( (o = super.next()) != null) { + List entry = (List) o; + long version = (Long) entry.get(UpdateLog.VERSION_IDX); + version = Math.abs(version); + versionToPos.put(version, pos); + pos = currentPos(); + + if (version < lastVersion) inOrder = false; + lastVersion = version; + } + fis.seek(startingPos); + } + + if (inOrder) { + return super.next(); + } else { + if (iterator == null) iterator = versionToPos.values().iterator(); + if (!iterator.hasNext()) return null; + long pos = iterator.next(); + if (pos != currentPos()) fis.seek(pos); + return super.next(); + } + } + } + public abstract class ReverseReader { /** Returns the next object from the log, or null if none available. diff --git a/solr/core/src/java/org/apache/solr/update/UpdateCommand.java b/solr/core/src/java/org/apache/solr/update/UpdateCommand.java index 9f015717027..b124271d977 100644 --- a/solr/core/src/java/org/apache/solr/update/UpdateCommand.java +++ b/solr/core/src/java/org/apache/solr/update/UpdateCommand.java @@ -34,6 +34,7 @@ public abstract class UpdateCommand implements Cloneable { public static int PEER_SYNC = 0x00000004; // update command is a missing update being provided by a peer. public static int IGNORE_AUTOCOMMIT = 0x00000008; // this update should not count toward triggering of autocommits. public static int CLEAR_CACHES = 0x00000010; // clear caches associated with the update log. used when applying reordered DBQ updates when doing an add. + public static int IGNORE_INDEXWRITER = 0x00000020; public UpdateCommand(SolrQueryRequest req) { this.req = req; diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java index 16eff9ceece..6a5f407daa1 100644 --- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java +++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java @@ -27,6 +27,7 @@ import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Deque; import java.util.HashMap; import java.util.LinkedHashMap; @@ -618,7 +619,7 @@ public static final int VERSION_IDX = 1; } // only change our caches if we are not buffering - if ((cmd.getFlags() & UpdateCommand.BUFFERING) == 0) { + if ((cmd.getFlags() & UpdateCommand.BUFFERING) == 0 && (cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) == 0) { // given that we just did a delete-by-query, we don't know what documents were // affected and hence we must purge our caches. openRealtimeSearcher(); @@ -1095,6 +1096,162 @@ public static final int VERSION_IDX = 1; return cs.submit(replayer, recoveryInfo); } + /** + * Replay current tlog, so all updates will be written to index. + * This is must do task for a append replica become a new leader. + * @return future of this task + */ + public Future recoverFromCurrentLog() { + if (tlog == null) { + return null; + } + map.clear(); + recoveryInfo = new RecoveryInfo(); + tlog.incref(); + + ExecutorCompletionService cs = new ExecutorCompletionService<>(recoveryExecutor); + LogReplayer replayer = new LogReplayer(Collections.singletonList(tlog), false, true); + + versionInfo.blockUpdates(); + try { + state = State.REPLAYING; + } finally { + versionInfo.unblockUpdates(); + } + + return cs.submit(replayer, recoveryInfo); + } + + /** + * Block updates, append a commit at current tlog, + * then copy over buffer updates to new tlog and bring back ulog to active state. + * So any updates which hasn't made it to the index is preserved in the current tlog, + * this also make RTG work + * @param cuc any updates that have version larger than the version of cuc will be copied over + */ + public void copyOverBufferingUpdates(CommitUpdateCommand cuc) { + versionInfo.blockUpdates(); + try { + operationFlags &= ~FLAG_GAP; + state = State.ACTIVE; + copyAndSwitchToNewTlog(cuc); + } finally { + versionInfo.unblockUpdates(); + } + } + + /** + * Block updates, append a commit at current tlog, then copy over updates to a new tlog. + * So any updates which hasn't made it to the index is preserved in the current tlog + * @param cuc any updates that have version larger than the version of cuc will be copied over + */ + public void copyOverOldUpdates(CommitUpdateCommand cuc) { + versionInfo.blockUpdates(); + try { + copyAndSwitchToNewTlog(cuc); + } finally { + versionInfo.unblockUpdates(); + } + } + + protected void copyAndSwitchToNewTlog(CommitUpdateCommand cuc) { + synchronized (this) { + if (tlog == null) return; + preCommit(cuc); + try { + copyOverOldUpdates(cuc.getVersion()); + } finally { + postCommit(cuc); + } + } + } + + /** + * Copy over updates from prevTlog or last tlog (in tlog folder) to a new tlog + * @param commitVersion any updates that have version larger than the commitVersion will be copied over + */ + public void copyOverOldUpdates(long commitVersion) { + TransactionLog oldTlog = prevTlog; + if (oldTlog == null && !logs.isEmpty()) { + oldTlog = logs.getFirst(); + } + if (oldTlog == null || oldTlog.refcount.get() == 0) { + return; + } + + try { + if (oldTlog.endsWithCommit()) { + return; + } + } catch (IOException e) { + log.warn("Exception reading log", e); + return; + } + + SolrQueryRequest req = new LocalSolrQueryRequest(uhandler.core, + new ModifiableSolrParams()); + TransactionLog.LogReader logReader = oldTlog.getReader(0); + Object o = null; + try { + while ( (o = logReader.next()) != null ) { + try { + List entry = (List)o; + int operationAndFlags = (Integer) entry.get(0); + int oper = operationAndFlags & OPERATION_MASK; + long version = (Long) entry.get(1); + if (Math.abs(version) > commitVersion) { + switch (oper) { + case UpdateLog.UPDATE_INPLACE: + case UpdateLog.ADD: { + SolrInputDocument sdoc = (SolrInputDocument) entry.get(entry.size() - 1); + AddUpdateCommand cmd = new AddUpdateCommand(req); + cmd.solrDoc = sdoc; + cmd.setVersion(version); + cmd.setFlags(UpdateCommand.IGNORE_AUTOCOMMIT); + add(cmd); + break; + } + case UpdateLog.DELETE: { + byte[] idBytes = (byte[]) entry.get(2); + DeleteUpdateCommand cmd = new DeleteUpdateCommand(req); + cmd.setIndexedId(new BytesRef(idBytes)); + cmd.setVersion(version); + cmd.setFlags(UpdateCommand.IGNORE_AUTOCOMMIT); + delete(cmd); + break; + } + + case UpdateLog.DELETE_BY_QUERY: { + String query = (String) entry.get(2); + DeleteUpdateCommand cmd = new DeleteUpdateCommand(req); + cmd.query = query; + cmd.setVersion(version); + cmd.setFlags(UpdateCommand.IGNORE_AUTOCOMMIT); + deleteByQuery(cmd); + break; + } + + default: + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown Operation! " + oper); + } + } + } catch (ClassCastException e) { + log.warn("Unexpected log entry or corrupt log. Entry=" + o, e); + } + } + // Prev tlog will be closed, so nullify prevMap + if (prevTlog == oldTlog) { + prevMap = null; + } + } catch (IOException e) { + log.error("Exception reading versions from log",e); + } catch (InterruptedException e) { + log.warn("Exception reading log", e); + } finally { + if (logReader != null) logReader.close(); + } + } + protected void ensureLog() { if (tlog == null) { @@ -1482,6 +1639,7 @@ public static final int VERSION_IDX = 1; boolean activeLog; boolean finishing = false; // state where we lock out other updates and finish those updates that snuck in before we locked boolean debug = loglog.isDebugEnabled(); + boolean inSortedOrder; public LogReplayer(List translogs, boolean activeLog) { this.translogs = new LinkedList<>(); @@ -1489,6 +1647,11 @@ public static final int VERSION_IDX = 1; this.activeLog = activeLog; } + public LogReplayer(List translogs, boolean activeLog, boolean inSortedOrder) { + this(translogs, activeLog); + this.inSortedOrder = inSortedOrder; + } + private SolrQueryRequest req; @@ -1554,7 +1717,11 @@ public static final int VERSION_IDX = 1; try { loglog.warn("Starting log replay " + translog + " active=" + activeLog + " starting pos=" + recoveryInfo.positionOfStart); long lastStatusTime = System.nanoTime(); - tlogReader = translog.getReader(recoveryInfo.positionOfStart); + if (inSortedOrder) { + tlogReader = translog.getSortedReader(recoveryInfo.positionOfStart); + } else { + tlogReader = translog.getReader(recoveryInfo.positionOfStart); + } // NOTE: we don't currently handle a core reload during recovery. This would cause the core // to change underneath us. diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java index ec093cf5dfe..08ede724269 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java +++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java @@ -279,6 +279,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { // this is set to true in the constructor if the next processors in the chain // are custom and may modify the SolrInputDocument racing with its serialization for replication private final boolean cloneRequiredOnLeader; + private final boolean onlyLeaderIndexes; public DistributedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) { this(req, rsp, new AtomicUpdateDocumentMerger(req), next); @@ -324,8 +325,12 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { if (cloudDesc != null) { collection = cloudDesc.getCollectionName(); + ClusterState cstate = zkController.getClusterState(); + DocCollection coll = cstate.getCollection(collection); + onlyLeaderIndexes = coll.getRealtimeReplicas() == 1; } else { collection = null; + onlyLeaderIndexes = false; } boolean shouldClone = false; @@ -1186,6 +1191,9 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { checkDeleteByQueries = true; } } + if (onlyLeaderIndexes && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) { + cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER); + } } } @@ -1692,6 +1700,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { return; } + if (onlyLeaderIndexes && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) { + cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER); + } + doLocalDelete(cmd); } } @@ -1845,6 +1857,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { return true; } } + + if (onlyLeaderIndexes && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) { + cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER); + } } } @@ -1876,7 +1892,27 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { } if (!zkEnabled || req.getParams().getBool(COMMIT_END_POINT, false) || singleLeader) { - doLocalCommit(cmd); + if (onlyLeaderIndexes) { + try { + Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry( + collection, cloudDesc.getShardId()); + isLeader = leaderReplica.getName().equals( + req.getCore().getCoreDescriptor().getCloudDescriptor() + .getCoreNodeName()); + if (isLeader) { + long commitVersion = vinfo.getNewClock(); + cmd.setVersion(commitVersion); + doLocalCommit(cmd); + } else { + assert TestInjection.waitForInSyncWithLeader(req.getCore(), + zkController, collection, cloudDesc.getShardId()); + } + } catch (InterruptedException e) { + throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Exception finding leader for shard " + cloudDesc.getShardId(), e); + } + } else { + doLocalCommit(cmd); + } } else { ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams())); if (!req.getParams().getBool(COMMIT_END_POINT, false)) { diff --git a/solr/core/src/java/org/apache/solr/util/TestInjection.java b/solr/core/src/java/org/apache/solr/util/TestInjection.java index 5e4dc75b11c..97291a1f6a7 100644 --- a/solr/core/src/java/org/apache/solr/util/TestInjection.java +++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java @@ -28,14 +28,28 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.solr.client.solrj.impl.HttpSolrClient; +import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.cloud.ZkController; import org.apache.solr.common.NonExistentCoreException; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; +import org.apache.solr.common.cloud.Replica; +import org.apache.solr.common.params.CommonParams; +import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.Pair; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.core.CoreContainer; +import org.apache.solr.core.SolrCore; +import org.apache.solr.handler.ReplicationHandler; +import org.apache.solr.update.SolrIndexWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.handler.ReplicationHandler.CMD_DETAILS; +import static org.apache.solr.handler.ReplicationHandler.COMMAND; + /** * Allows random faults to be injected in running code during test runs. @@ -118,6 +132,8 @@ public class TestInjection { public static int randomDelayMaxInCoreCreationInSec = 10; public static String splitFailureBeforeReplicaCreation = null; + + public static String waitForReplicasInSync = "true:60"; private static Set timers = Collections.synchronizedSet(new HashSet()); @@ -343,6 +359,44 @@ public class TestInjection { return true; } + + @SuppressForbidden(reason = "Need currentTimeMillis, because COMMIT_TIME_MSEC_KEY use currentTimeMillis as value") + public static boolean waitForInSyncWithLeader(SolrCore core, ZkController zkController, String collection, String shardId) throws InterruptedException { + if (waitForReplicasInSync == null) return true; + + Pair pair = parseValue(waitForReplicasInSync); + boolean enabled = pair.first(); + if (!enabled) return true; + long t = System.currentTimeMillis() - 100; + try { + for (int i = 0; i < pair.second(); i++) { + if (core.isClosed()) return true; + Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry( + collection, shardId); + try (HttpSolrClient leaderClient = new HttpSolrClient.Builder(leaderReplica.getCoreUrl()).build()) { + ModifiableSolrParams params = new ModifiableSolrParams(); + params.set(CommonParams.QT, ReplicationHandler.PATH); + params.set(COMMAND, CMD_DETAILS); + + NamedList response = leaderClient.request(new QueryRequest(params)); + long leaderVersion = (long) ((NamedList)response.get("details")).get("indexVersion"); + + String localVersion = core.getDeletionPolicy().getLatestCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY); + if (localVersion == null && leaderVersion == 0 && !core.getUpdateHandler().getUpdateLog().hasUncommittedChanges()) return true; + if (localVersion != null && Long.parseLong(localVersion) == leaderVersion && (leaderVersion >= t || i >= 6)) { + return true; + } else { + Thread.sleep(500); + } + } + } + + } catch (Exception e) { + log.error("Exception when wait for replicas in sync with master"); + } + + return false; + } private static Pair parseValue(String raw) { Matcher m = ENABLED_PERCENT.matcher(raw); diff --git a/solr/core/src/test-files/solr/configsets/cloud-minimal-inplace-updates/conf/schema.xml b/solr/core/src/test-files/solr/configsets/cloud-minimal-inplace-updates/conf/schema.xml new file mode 100644 index 00000000000..31802f91bb2 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/cloud-minimal-inplace-updates/conf/schema.xml @@ -0,0 +1,31 @@ + + + + + + + + + + + id + + + + + diff --git a/solr/core/src/test-files/solr/configsets/cloud-minimal-inplace-updates/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cloud-minimal-inplace-updates/conf/solrconfig.xml new file mode 100644 index 00000000000..8da7d2847e9 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/cloud-minimal-inplace-updates/conf/solrconfig.xml @@ -0,0 +1,48 @@ + + + + + + + + + ${solr.data.dir:} + + + + + ${tests.luceneMatchVersion:LATEST} + + + + ${solr.commitwithin.softcommit:true} + + + + + + + explicit + true + text + + + + + diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java index 582c8b402ea..5eb4b3b35d9 100644 --- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java +++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java @@ -54,11 +54,17 @@ public class BasicDistributedZk2Test extends AbstractFullDistribZkTestBase { private static final String SHARD2 = "shard2"; private static final String SHARD1 = "shard1"; private static final String ONE_NODE_COLLECTION = "onenodecollection"; + private final boolean onlyLeaderIndexes = random().nextBoolean(); public BasicDistributedZk2Test() { super(); sliceCount = 2; } + + @Override + protected int getRealtimeReplicas() { + return onlyLeaderIndexes? 1 : -1; + } @Test @ShardsFixed(num = 4) diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java index 25c483bb9e8..d1dbe9c4319 100644 --- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java @@ -87,6 +87,8 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final String DEFAULT_COLLECTION = "collection1"; + + private final boolean onlyLeaderIndexes = random().nextBoolean(); String t1="a_t"; String i1="a_i1"; String tlong = "other_tl1"; @@ -114,7 +116,12 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase { pending = new HashSet<>(); } - + + @Override + protected int getRealtimeReplicas() { + return onlyLeaderIndexes? 1 : -1; + } + @Override protected void setDistributedParams(ModifiableSolrParams params) { diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java index 4e6122e8fe0..628884c3cda 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java @@ -55,6 +55,8 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1")); + private final boolean onlyLeaderIndexes = random().nextBoolean(); + @BeforeClass public static void beforeSuperClass() { schemaString = "schema15.xml"; // we need a string id @@ -109,6 +111,11 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase clientSoTimeout = 5000; } + @Override + protected int getRealtimeReplicas() { + return onlyLeaderIndexes? 1 : -1; + } + @Test public void test() throws Exception { cloudClient.setSoTimeout(clientSoTimeout); diff --git a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java index e9e8907affa..8904ea827be 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java @@ -55,6 +55,12 @@ import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP; public class ForceLeaderTest extends HttpPartitionTest { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private final boolean onlyLeaderIndexes = random().nextBoolean(); + + @Override + protected int getRealtimeReplicas() { + return onlyLeaderIndexes? 1 : -1; + } @Test @Override diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java index 5ae4c17c9de..01002cfee44 100644 --- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java @@ -76,12 +76,19 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase { // give plenty of time for replicas to recover when running in slow Jenkins test envs protected static final int maxWaitSecsToSeeAllActive = 90; + private final boolean onlyLeaderIndexes = random().nextBoolean(); + public HttpPartitionTest() { super(); sliceCount = 2; fixShardCount(3); } + @Override + protected int getRealtimeReplicas() { + return onlyLeaderIndexes? 1 : -1; + } + /** * We need to turn off directUpdatesToLeadersOnly due to SOLR-9512 */ diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java index fd122ad6a08..457b9d9ef85 100644 --- a/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java @@ -37,12 +37,19 @@ public class LeaderInitiatedRecoveryOnCommitTest extends BasicDistributedZkTest private static final long sleepMsBeforeHealPartition = 2000L; + private final boolean onlyLeaderIndexes = random().nextBoolean(); + public LeaderInitiatedRecoveryOnCommitTest() { super(); sliceCount = 1; fixShardCount(4); } + @Override + protected int getRealtimeReplicas() { + return onlyLeaderIndexes? 1 : -1; + } + @Override @Test public void test() throws Exception { diff --git a/solr/core/src/test/org/apache/solr/cloud/OnlyLeaderIndexesTest.java b/solr/core/src/test/org/apache/solr/cloud/OnlyLeaderIndexesTest.java new file mode 100644 index 00000000000..a4e8d6f2bc4 --- /dev/null +++ b/solr/core/src/test/org/apache/solr/cloud/OnlyLeaderIndexesTest.java @@ -0,0 +1,435 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.solr.cloud; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Semaphore; + +import org.apache.lucene.index.IndexWriter; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.embedded.JettySolrRunner; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.cloud.DocCollection; +import org.apache.solr.common.cloud.Replica; +import org.apache.solr.common.cloud.Slice; +import org.apache.solr.common.cloud.ZkNodeProps; +import org.apache.solr.common.cloud.ZkStateReader; +import org.apache.solr.core.SolrCore; +import org.apache.solr.update.DirectUpdateHandler2; +import org.apache.solr.update.SolrIndexWriter; +import org.apache.solr.update.UpdateHandler; +import org.apache.solr.update.UpdateLog; +import org.apache.solr.util.RefCounted; +import org.junit.BeforeClass; +import org.junit.Test; + +public class OnlyLeaderIndexesTest extends SolrCloudTestCase { + private static final String COLLECTION = "collection1"; + + @BeforeClass + public static void setupCluster() throws Exception { + System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory"); + System.setProperty("solr.ulog.numRecordsToKeep", "1000"); + + configureCluster(3) + .addConfig("config", TEST_PATH().resolve("configsets") + .resolve("cloud-minimal-inplace-updates").resolve("conf")) + .configure(); + + CollectionAdminRequest + .createCollection(COLLECTION, "config", 1, 3) + .setRealtimeReplicas(1) + .setMaxShardsPerNode(1) + .process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, 30); + } + + @Test + public void test() throws Exception { + basicTest(); + recoveryTest(); + dbiTest(); + basicLeaderElectionTest(); + outOfOrderDBQWithInPlaceUpdatesTest(); + } + + public void basicTest() throws Exception { + CloudSolrClient cloudClient = cluster.getSolrClient(); + new UpdateRequest() + .add(sdoc("id", "1")) + .add(sdoc("id", "2")) + .add(sdoc("id", "3")) + .add(sdoc("id", "4")) + .process(cloudClient, COLLECTION); + + { + UpdateHandler updateHandler = getSolrCore(true).get(0).getUpdateHandler(); + RefCounted iwRef = updateHandler.getSolrCoreState().getIndexWriter(null); + assertTrue("IndexWriter at leader must see updates ", iwRef.get().hasUncommittedChanges()); + iwRef.decref(); + } + + for (SolrCore solrCore : getSolrCore(false)) { + RefCounted iwRef = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(null); + assertFalse("IndexWriter at replicas must not see updates ", iwRef.get().hasUncommittedChanges()); + iwRef.decref(); + } + + checkRTG(1, 4, cluster.getJettySolrRunners()); + + new UpdateRequest() + .deleteById("1") + .deleteByQuery("id:2") + .process(cloudClient, COLLECTION); + + // The DBQ is not processed at replicas, so we still can get doc2 and other docs by RTG + checkRTG(2,4, getSolrRunner(false)); + + new UpdateRequest() + .commit(cloudClient, COLLECTION); + + checkShardConsistency(2, 1); + + // Update log roll over + for (SolrCore solrCore : getSolrCore(false)) { + UpdateLog updateLog = solrCore.getUpdateHandler().getUpdateLog(); + assertFalse(updateLog.hasUncommittedChanges()); + } + + // UpdateLog copy over old updates + for (int i = 15; i <= 150; i++) { + cloudClient.add(COLLECTION, sdoc("id",String.valueOf(i))); + if (random().nextInt(100) < 15 & i != 150) { + cloudClient.commit(COLLECTION); + } + } + checkRTG(120,150, cluster.getJettySolrRunners()); + waitForReplicasCatchUp(20); + } + + public void recoveryTest() throws Exception { + CloudSolrClient cloudClient = cluster.getSolrClient(); + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); + new UpdateRequest() + .add(sdoc("id", "3")) + .add(sdoc("id", "4")) + .commit(cloudClient, COLLECTION); + // Replica recovery + new UpdateRequest() + .add(sdoc("id", "5")) + .process(cloudClient, COLLECTION); + JettySolrRunner solrRunner = getSolrRunner(false).get(0); + ChaosMonkey.stop(solrRunner); + new UpdateRequest() + .add(sdoc("id", "6")) + .process(cloudClient, COLLECTION); + ChaosMonkey.start(solrRunner); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, 30); + // We skip peerSync, so replica will always trigger commit on leader + checkShardConsistency(4, 20); + + // LTR can be kicked off, so waiting for replicas recovery + new UpdateRequest() + .add(sdoc("id", "7")) + .commit(cloudClient, COLLECTION); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, 30); + checkShardConsistency(5, 20); + + // More Replica recovery testing + new UpdateRequest() + .add(sdoc("id", "8")) + .process(cloudClient, COLLECTION); + checkRTG(3,8, cluster.getJettySolrRunners()); + DirectUpdateHandler2.commitOnClose = false; + ChaosMonkey.stop(solrRunner); + DirectUpdateHandler2.commitOnClose = true; + ChaosMonkey.start(solrRunner); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, 30); + checkRTG(3,8, cluster.getJettySolrRunners()); + checkShardConsistency(6, 20); + + // Test replica recovery apply buffer updates + Semaphore waitingForBufferUpdates = new Semaphore(0); + Semaphore waitingForReplay = new Semaphore(0); + RecoveryStrategy.testing_beforeReplayBufferingUpdates = () -> { + try { + waitingForReplay.release(); + waitingForBufferUpdates.acquire(); + } catch (InterruptedException e) { + e.printStackTrace(); + } + }; + ChaosMonkey.stop(solrRunner); + ChaosMonkey.start(solrRunner); + waitingForReplay.acquire(); + new UpdateRequest() + .add(sdoc("id", "9")) + .add(sdoc("id", "10")) + .process(cloudClient, COLLECTION); + waitingForBufferUpdates.release(); + RecoveryStrategy.testing_beforeReplayBufferingUpdates = null; + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, 30); + checkRTG(3,10, cluster.getJettySolrRunners()); + checkShardConsistency(6, 20); + for (SolrCore solrCore : getSolrCore(false)) { + RefCounted iwRef = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(null); + assertFalse("IndexWriter at replicas must not see updates ", iwRef.get().hasUncommittedChanges()); + iwRef.decref(); + } + } + + public void dbiTest() throws Exception{ + CloudSolrClient cloudClient = cluster.getSolrClient(); + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); + new UpdateRequest() + .add(sdoc("id", "1")) + .commit(cloudClient, COLLECTION); + checkShardConsistency(1, 1); + new UpdateRequest() + .deleteById("1") + .process(cloudClient, COLLECTION); + try { + checkRTG(1, 1, cluster.getJettySolrRunners()); + } catch (AssertionError e) { + return; + } + fail("Doc1 is deleted but it's still exist"); + } + + public void basicLeaderElectionTest() throws Exception { + CloudSolrClient cloudClient = cluster.getSolrClient(); + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); + new UpdateRequest() + .add(sdoc("id", "1")) + .add(sdoc("id", "2")) + .process(cloudClient, COLLECTION); + String oldLeader = getLeader(); + JettySolrRunner oldLeaderJetty = getSolrRunner(true).get(0); + ChaosMonkey.kill(oldLeaderJetty); + for (int i = 0; i < 60; i++) { // wait till leader is changed + if (!oldLeader.equals(getLeader())) { + break; + } + Thread.sleep(100); + } + new UpdateRequest() + .add(sdoc("id", "3")) + .add(sdoc("id", "4")) + .process(cloudClient, COLLECTION); + ChaosMonkey.start(oldLeaderJetty); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, 60); + checkRTG(1,4, cluster.getJettySolrRunners()); + new UpdateRequest() + .commit(cloudClient, COLLECTION); + checkShardConsistency(4,1); + } + + private String getLeader() throws InterruptedException { + ZkNodeProps props = cluster.getSolrClient().getZkStateReader().getLeaderRetry("collection1", "shard1", 30000); + return props.getStr(ZkStateReader.NODE_NAME_PROP); + } + + public void outOfOrderDBQWithInPlaceUpdatesTest() throws Exception { + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); + List updates = new ArrayList<>(); + updates.add(simulatedUpdateRequest(null, "id", 1, "title_s", "title0_new", "inplace_updatable_int", 5, "_version_", Long.MAX_VALUE-100)); // full update + updates.add(simulatedDBQ("inplace_updatable_int:5", Long.MAX_VALUE-98)); + updates.add(simulatedUpdateRequest(Long.MAX_VALUE-100, "id", 1, "inplace_updatable_int", 6, "_version_", Long.MAX_VALUE-99)); + for (JettySolrRunner solrRunner: getSolrRunner(false)) { + try (SolrClient client = solrRunner.newClient()) { + for (UpdateRequest up : updates) { + up.process(client, COLLECTION); + } + } + } + JettySolrRunner oldLeaderJetty = getSolrRunner(true).get(0); + ChaosMonkey.kill(oldLeaderJetty); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, 30); + ChaosMonkey.start(oldLeaderJetty); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, 30); + new UpdateRequest() + .add(sdoc("id", "2")) + .commit(cluster.getSolrClient(), COLLECTION); + checkShardConsistency(2,20); + SolrDocument doc = cluster.getSolrClient().getById(COLLECTION,"1"); + assertNotNull(doc.get("title_s")); + } + + private UpdateRequest simulatedUpdateRequest(Long prevVersion, Object... fields) throws SolrServerException, IOException { + SolrInputDocument doc = sdoc(fields); + + // get baseUrl of the leader + String baseUrl = getBaseUrl(); + + UpdateRequest ur = new UpdateRequest(); + ur.add(doc); + ur.setParam("update.distrib", "FROMLEADER"); + if (prevVersion != null) { + ur.setParam("distrib.inplace.prevversion", String.valueOf(prevVersion)); + ur.setParam("distrib.inplace.update", "true"); + } + ur.setParam("distrib.from", baseUrl); + return ur; + } + + private UpdateRequest simulatedDBQ(String query, long version) throws SolrServerException, IOException { + String baseUrl = getBaseUrl(); + + UpdateRequest ur = new UpdateRequest(); + ur.deleteByQuery(query); + ur.setParam("_version_", ""+version); + ur.setParam("update.distrib", "FROMLEADER"); + ur.setParam("distrib.from", baseUrl); + return ur; + } + + private String getBaseUrl() { + DocCollection collection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION); + Slice slice = collection.getSlice("shard1"); + return slice.getLeader().getCoreUrl(); + } + + private void checkRTG(int from, int to, List solrRunners) throws Exception{ + + for (JettySolrRunner solrRunner: solrRunners) { + try (SolrClient client = solrRunner.newClient()) { + for (int i = from; i <= to; i++) { + SolrQuery query = new SolrQuery("*:*"); + query.set("distrib", false); + query.setRequestHandler("/get"); + query.set("id",i); + QueryResponse res = client.query(COLLECTION, query); + assertNotNull("Can not find doc "+ i + " in " + solrRunner.getBaseUrl(),res.getResponse().get("doc")); + } + } + } + + } + + private void checkShardConsistency(int expected, int numTry) throws Exception{ + + for (int i = 0; i < numTry; i++) { + boolean inSync = true; + for (JettySolrRunner solrRunner: cluster.getJettySolrRunners()) { + try (SolrClient client = solrRunner.newClient()) { + SolrQuery query = new SolrQuery("*:*"); + query.set("distrib", false); + long results = client.query(COLLECTION, query).getResults().getNumFound(); + if (expected != results) { + inSync = false; + Thread.sleep(500); + break; + } + } + } + if (inSync) return; + } + + fail("Some replicas are not in sync with leader"); + } + + private void waitForReplicasCatchUp(int numTry) throws IOException, InterruptedException { + String leaderTimeCommit = getSolrCore(true).get(0).getDeletionPolicy().getLatestCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY); + if (leaderTimeCommit == null) return; + for (int i = 0; i < numTry; i++) { + boolean inSync = true; + for (SolrCore solrCore : getSolrCore(false)) { + String replicateTimeCommit = solrCore.getDeletionPolicy().getLatestCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY); + if (!leaderTimeCommit.equals(replicateTimeCommit)) { + inSync = false; + Thread.sleep(500); + break; + } + } + if (inSync) return; + } + + fail("Some replicas are not in sync with leader"); + + } + + private List getSolrCore(boolean isLeader) { + List rs = new ArrayList<>(); + + CloudSolrClient cloudClient = cluster.getSolrClient(); + DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION); + + for (JettySolrRunner solrRunner : cluster.getJettySolrRunners()) { + if (solrRunner.getCoreContainer() == null) continue; + for (SolrCore solrCore : solrRunner.getCoreContainer().getCores()) { + CloudDescriptor cloudDescriptor = solrCore.getCoreDescriptor().getCloudDescriptor(); + Slice slice = docCollection.getSlice(cloudDescriptor.getShardId()); + Replica replica = docCollection.getReplica(cloudDescriptor.getCoreNodeName()); + if (slice.getLeader() == replica && isLeader) { + rs.add(solrCore); + } else if (slice.getLeader() != replica && !isLeader) { + rs.add(solrCore); + } + } + } + return rs; + } + + private List getSolrRunner(boolean isLeader) { + List rs = new ArrayList<>(); + + CloudSolrClient cloudClient = cluster.getSolrClient(); + DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION); + + for (JettySolrRunner solrRunner : cluster.getJettySolrRunners()) { + if (solrRunner.getCoreContainer() == null) continue; + for (SolrCore solrCore : solrRunner.getCoreContainer().getCores()) { + CloudDescriptor cloudDescriptor = solrCore.getCoreDescriptor().getCloudDescriptor(); + Slice slice = docCollection.getSlice(cloudDescriptor.getShardId()); + Replica replica = docCollection.getReplica(cloudDescriptor.getCoreNodeName()); + if (slice.getLeader() == replica && isLeader) { + rs.add(solrRunner); + } else if (slice.getLeader() != replica && !isLeader) { + rs.add(solrRunner); + } + } + } + return rs; + } + +} \ No newline at end of file diff --git a/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java b/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java index c987c90571e..a8e14bf5465 100644 --- a/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java @@ -33,12 +33,17 @@ import org.junit.Test; @SolrTestCaseJ4.SuppressSSL public class RecoveryAfterSoftCommitTest extends AbstractFullDistribZkTestBase { private static final int MAX_BUFFERED_DOCS = 2, ULOG_NUM_RECORDS_TO_KEEP = 2; - + private final boolean onlyLeaderIndexes = random().nextBoolean(); public RecoveryAfterSoftCommitTest() { sliceCount = 1; fixShardCount(2); } + @Override + protected int getRealtimeReplicas() { + return onlyLeaderIndexes? 1: -1; + } + @BeforeClass public static void beforeTests() { System.setProperty("solr.tests.maxBufferedDocs", String.valueOf(MAX_BUFFERED_DOCS)); diff --git a/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java b/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java index 72f06943cf7..bf9b5e014c8 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java @@ -86,6 +86,12 @@ public class ShardSplitTest extends BasicDistributedZkTest { useFactory(null); } + //TODO for now, onlyLeaderIndexes do not work with ShardSplitTest + @Override + protected int getRealtimeReplicas() { + return -1; + } + @Test public void test() throws Exception { diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java index 1af09f4e817..b592861fb30 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java @@ -52,6 +52,7 @@ import org.junit.Test; public class TestCloudRecovery extends SolrCloudTestCase { private static final String COLLECTION = "collection1"; + private static boolean onlyLeaderIndexes; @BeforeClass public static void setupCluster() throws Exception { @@ -63,8 +64,10 @@ public class TestCloudRecovery extends SolrCloudTestCase { .addConfig("config", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf")) .configure(); + onlyLeaderIndexes = random().nextBoolean(); CollectionAdminRequest .createCollection(COLLECTION, "config", 2, 2) + .setRealtimeReplicas(onlyLeaderIndexes? 1: -1) .setMaxShardsPerNode(2) .process(cluster.getSolrClient()); AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), @@ -107,7 +110,12 @@ public class TestCloudRecovery extends SolrCloudTestCase { resp = cloudClient.query(COLLECTION, params); assertEquals(4, resp.getResults().getNumFound()); // Make sure all nodes is recover from tlog - assertEquals(4, countReplayLog.get()); + if (onlyLeaderIndexes) { + // Leader election can be kicked off, so 2 append replicas will replay its tlog before becoming new leader + assertTrue( countReplayLog.get() >=2); + } else { + assertEquals(4, countReplayLog.get()); + } // check metrics int replicationCount = 0; @@ -127,7 +135,11 @@ public class TestCloudRecovery extends SolrCloudTestCase { skippedCount += skipped.getCount(); } } - assertEquals(2, replicationCount); + if (onlyLeaderIndexes) { + assertTrue(replicationCount >= 2); + } else { + assertEquals(2, replicationCount); + } } @Test diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java index 8905077e4c8..8fbfee391e0 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java @@ -60,7 +60,10 @@ public class TestCollectionAPI extends ReplicaPropertiesBase { @ShardsFixed(num = 2) public void test() throws Exception { try (CloudSolrClient client = createCloudClient(null)) { - createCollection(null, COLLECTION_NAME, 2, 2, 2, client, null, "conf1"); + CollectionAdminRequest.Create req = CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf1",2,2); + req.setRealtimeReplicas(1); + req.setMaxShardsPerNode(2); + client.request(req); createCollection(null, COLLECTION_NAME1, 1, 1, 1, client, null, "conf1"); } @@ -170,6 +173,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase { Map collection = (Map) collections.get(COLLECTION_NAME); assertNotNull(collection); assertEquals("conf1", collection.get("configName")); + assertEquals("1", collection.get("realtimeReplicas")); } } diff --git a/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java index 97be8233324..1bba523c794 100644 --- a/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java @@ -42,7 +42,12 @@ public class HdfsBasicDistributedZkTest extends BasicDistributedZkTest { System.setProperty("tests.hdfs.numdatanodes", "1"); dfsCluster = HdfsTestUtil.setupClass(createTempDir().toFile().getAbsolutePath()); } - + + @Override + protected int getRealtimeReplicas() { + return -1; + } + @AfterClass public static void teardownClass() throws Exception { HdfsTestUtil.teardownClass(dfsCluster); diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java index bb0ab9adeea..7a4fa860e76 100644 --- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java +++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java @@ -71,6 +71,7 @@ import org.slf4j.LoggerFactory; @Slow public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private final boolean onlyLeaderIndexes = random().nextBoolean(); @BeforeClass public static void beforeSuperClass() throws Exception { @@ -108,7 +109,12 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { iw.decref(); } } - + + @Override + protected int getRealtimeReplicas() { + return onlyLeaderIndexes? 1 : -1; + } + @After public void after() { System.clearProperty("solr.tests.intClassName"); @@ -265,6 +271,10 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { } private void reorderedDBQIndividualReplicaTest() throws Exception { + if (onlyLeaderIndexes) { + log.info("RTG with DBQs are not working in append replicas"); + return; + } clearIndex(); commit(); @@ -595,7 +605,6 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { } private void outOfOrderUpdatesIndividualReplicaTest() throws Exception { - clearIndex(); commit(); @@ -741,6 +750,10 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { DV(id=x, val=5, ver=3) */ private void reorderedDBQsResurrectionTest() throws Exception { + if (onlyLeaderIndexes) { + log.info("RTG with DBQs are not working in append replicas"); + return; + } clearIndex(); commit(); @@ -1016,7 +1029,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { String baseUrl = getBaseUrl(""+id); UpdateRequest ur = new UpdateRequest(); - if (random().nextBoolean()) { + if (random().nextBoolean() || onlyLeaderIndexes) { ur.deleteById(""+id); } else { ur.deleteByQuery("id:"+id); @@ -1138,6 +1151,10 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { * dbq("inp:14",version=4) */ private void reorderedDBQsUsingUpdatedValueFromADroppedUpdate() throws Exception { + if (onlyLeaderIndexes) { + log.info("RTG with DBQs are not working in append replicas"); + return; + } clearIndex(); commit(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java index 94750c0acae..8beb6ed3c12 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java @@ -366,6 +366,7 @@ public abstract class CollectionAdminRequest private Properties properties; protected Boolean autoAddReplicas; + protected Integer realtimeReplicas; protected Integer stateFormat; private String[] rule , snitch; @@ -407,6 +408,7 @@ public abstract class CollectionAdminRequest public Create setNumShards(Integer numShards) {this.numShards = numShards; return this; } public Create setMaxShardsPerNode(Integer numShards) { this.maxShardsPerNode = numShards; return this; } public Create setAutoAddReplicas(boolean autoAddReplicas) { this.autoAddReplicas = autoAddReplicas; return this; } + public Create setRealtimeReplicas(Integer realtimeReplicas) { this.realtimeReplicas = realtimeReplicas; return this;} @Deprecated public Create setReplicationFactor(Integer repl) { this.replicationFactor = repl; return this; } public Create setStateFormat(Integer stateFormat) { this.stateFormat = stateFormat; return this; } @@ -421,6 +423,7 @@ public abstract class CollectionAdminRequest public Integer getMaxShardsPerNode() { return maxShardsPerNode; } public Integer getReplicationFactor() { return replicationFactor; } public Boolean getAutoAddReplicas() { return autoAddReplicas; } + public Integer getRealtimeReplicas() { return realtimeReplicas; } public Integer getStateFormat() { return stateFormat; } /** @@ -507,6 +510,9 @@ public abstract class CollectionAdminRequest if (autoAddReplicas != null) { params.set(ZkStateReader.AUTO_ADD_REPLICAS, autoAddReplicas); } + if (realtimeReplicas != null) { + params.set(ZkStateReader.REALTIME_REPLICAS, realtimeReplicas); + } if(properties != null) { addProperties(params, properties); } diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java index 179b9d54121..bf0f04ff579 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java @@ -33,6 +33,7 @@ import org.noggit.JSONWriter; import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS; import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE; +import static org.apache.solr.common.cloud.ZkStateReader.REALTIME_REPLICAS; import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR; /** @@ -59,6 +60,7 @@ public class DocCollection extends ZkNodeProps implements Iterable { private final Integer replicationFactor; private final Integer maxShardsPerNode; private final Boolean autoAddReplicas; + private final Integer realtimeReplicas; public DocCollection(String name, Map slices, Map props, DocRouter router) { @@ -84,6 +86,11 @@ public class DocCollection extends ZkNodeProps implements Iterable { this.maxShardsPerNode = (Integer) verifyProp(props, MAX_SHARDS_PER_NODE); Boolean autoAddReplicas = (Boolean) verifyProp(props, AUTO_ADD_REPLICAS); this.autoAddReplicas = autoAddReplicas == null ? false : autoAddReplicas; + Integer realtimeReplicas = (Integer) verifyProp(props, REALTIME_REPLICAS); + this.realtimeReplicas = realtimeReplicas == null ? -1 : realtimeReplicas; + if (this.realtimeReplicas != -1 && this.realtimeReplicas != 1) { + throw new SolrException(ErrorCode.SERVER_ERROR, "Invalid realtimeReplicas must be 1 or -1, found:" + this.realtimeReplicas); + } verifyProp(props, RULE); verifyProp(props, SNITCH); Iterator> iter = slices.entrySet().iterator(); @@ -126,6 +133,7 @@ public class DocCollection extends ZkNodeProps implements Iterable { switch (propName) { case MAX_SHARDS_PER_NODE: case REPLICATION_FACTOR: + case REALTIME_REPLICAS: return Integer.parseInt(o.toString()); case AUTO_ADD_REPLICAS: return Boolean.parseBoolean(o.toString()); @@ -226,6 +234,10 @@ public class DocCollection extends ZkNodeProps implements Iterable { return maxShardsPerNode; } + public int getRealtimeReplicas() { + return realtimeReplicas; + } + public String getZNode(){ return znode; } diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java index fea59780f8c..51b4b59d806 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java @@ -96,6 +96,7 @@ public class ZkStateReader implements Closeable { public static final String MAX_SHARDS_PER_NODE = "maxShardsPerNode"; public static final String AUTO_ADD_REPLICAS = "autoAddReplicas"; public static final String MAX_CORES_PER_NODE = "maxCoresPerNode"; + public static final String REALTIME_REPLICAS = "realtimeReplicas"; public static final String ROLES = "/roles.json"; diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java index 04eb722d945..ade1c699af0 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java @@ -272,6 +272,10 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes chaosMonkey = new ChaosMonkey(zkServer, zkStateReader, DEFAULT_COLLECTION, shardToJetty, shardToLeaderJetty); } + + protected int getRealtimeReplicas() { + return -1; + } protected CloudSolrClient createCloudClient(String defaultCollection) { CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean()); @@ -383,7 +387,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes Utils.toJSON(Utils.makeMap(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.CREATE.toLower(), "name", DEFAULT_COLLECTION, "numShards", String.valueOf(sliceCount), - DocCollection.STATE_FORMAT, getStateFormat()))); + DocCollection.STATE_FORMAT, getStateFormat(), + ZkStateReader.REALTIME_REPLICAS, getRealtimeReplicas()))); zkClient.close(); } @@ -1619,7 +1624,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes NUM_SLICES, numShards, ZkStateReader.REPLICATION_FACTOR, replicationFactor, CREATE_NODE_SET, createNodeSetStr, - ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode), + ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode, + ZkStateReader.REALTIME_REPLICAS, getRealtimeReplicas()), client); } @@ -1631,7 +1637,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes NUM_SLICES, numShards, ZkStateReader.REPLICATION_FACTOR, replicationFactor, CREATE_NODE_SET, createNodeSetStr, - ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode), + ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode, + ZkStateReader.REALTIME_REPLICAS, getRealtimeReplicas()), client, configName); } @@ -1814,6 +1821,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes Map props = makeMap( ZkStateReader.REPLICATION_FACTOR, replicationFactor, ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode, + ZkStateReader.REALTIME_REPLICAS, getRealtimeReplicas(), NUM_SLICES, numShards); Map> collectionInfos = new HashMap<>(); createCollection(collectionInfos, collName, props, client); From 83772c6f995604d854d3e6b0d9ff36c8eb2599c6 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Tue, 14 Mar 2017 16:20:46 +0700 Subject: [PATCH 02/20] SOLR-9835: Update CHANGES.txt --- solr/CHANGES.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 80ecea836ea..6f6516f791e 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -56,6 +56,8 @@ New Features ---------------------- * SOLR-9857, SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer. (ab) +* SOLR-9835: Create another replication mode for SolrCloud + Bug Fixes ---------------------- * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509. From cab7e1a3d7027ba13f49aa69d4eb473609c42dc9 Mon Sep 17 00:00:00 2001 From: markrmiller Date: Tue, 14 Mar 2017 06:01:06 -0400 Subject: [PATCH 03/20] SOLR-10279: The autoAddReplica feature can result in SolrCores being assigned new shards when using legacyCloud=false and will also fail on a state check when taking over a core registration with a new core. --- solr/CHANGES.txt | 4 +++ .../OverseerAutoReplicaFailoverThread.java | 6 ++-- .../org/apache/solr/cloud/ZkController.java | 8 +---- .../SharedFSAutoReplicaFailoverTest.java | 29 +++++++++++++++++++ 4 files changed, 38 insertions(+), 9 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 6f6516f791e..280f165a775 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -243,6 +243,10 @@ Bug Fixes * SOLR-10269: MetricsHandler JSON output incorrect. (ab) +* SOLR-10279: The autoAddReplica feature can result in SolrCores being assigned new shards when using + legacyCloud=false and will also fail on a state check when taking over a core registration with a new + core. (Mark Miller, Hrishikesh Gadre, Patrick Dvorack) + Optimizations ---------------------- diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java index 10b4bf3fb0e..ea09eeff213 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java @@ -243,13 +243,14 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable { final String dataDir = badReplica.replica.getStr("dataDir"); final String ulogDir = badReplica.replica.getStr("ulogDir"); final String coreNodeName = badReplica.replica.getName(); + final String shardId = badReplica.slice.getName(); if (dataDir != null) { // need an async request - full shard goes down leader election final String coreName = badReplica.replica.getStr(ZkStateReader.CORE_NAME_PROP); log.debug("submit call to {}", createUrl); MDC.put("OverseerAutoReplicaFailoverThread.createUrl", createUrl); try { - updateExecutor.submit(() -> createSolrCore(collection, createUrl, dataDir, ulogDir, coreNodeName, coreName)); + updateExecutor.submit(() -> createSolrCore(collection, createUrl, dataDir, ulogDir, coreNodeName, coreName, shardId)); } finally { MDC.remove("OverseerAutoReplicaFailoverThread.createUrl"); } @@ -440,7 +441,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable { private boolean createSolrCore(final String collection, final String createUrl, final String dataDir, final String ulogDir, - final String coreNodeName, final String coreName) { + final String coreNodeName, final String coreName, final String shardId) { try (HttpSolrClient client = new HttpSolrClient.Builder(createUrl).build()) { log.debug("create url={}", createUrl); @@ -451,6 +452,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable { createCmd.setCoreNodeName(coreNodeName); // TODO: how do we ensure unique coreName // for now, the collections API will use unique names + createCmd.setShardId(shardId); createCmd.setCoreName(coreName); createCmd.setDataDir(dataDir); createCmd.setUlogDir(ulogDir.substring(0, ulogDir.length() - "/tlog".length())); diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java index a19b35111b1..69a77f94820 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -1449,13 +1449,7 @@ public class ZkController { errorMessage.set("coreNodeName " + coreNodeName + " does not exist in shard " + cloudDesc.getShardId()); return false; } - String baseUrl = replica.getStr(BASE_URL_PROP); - String coreName = replica.getStr(CORE_NAME_PROP); - if (baseUrl.equals(this.baseURL) && coreName.equals(cd.getName())) { - return true; - } - errorMessage.set("coreNodeName " + coreNodeName + " exists, but does not match expected node or core name"); - return false; + return true; }); } catch (TimeoutException e) { String error = errorMessage.get(); diff --git a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java index 18503e76bfa..9c345fdf979 100644 --- a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java @@ -38,6 +38,7 @@ import org.apache.solr.SolrTestCaseJ4.SuppressSSL; import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create; import org.apache.solr.client.solrj.request.QueryRequest; import org.apache.solr.client.solrj.response.CollectionAdminResponse; @@ -103,6 +104,11 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa public void setUp() throws Exception { super.setUp(); collectionUlogDirMap.clear(); + if (random().nextBoolean()) { + CollectionAdminRequest.setClusterProperty("legacyCloud", "false").process(cloudClient); + } else { + CollectionAdminRequest.setClusterProperty("legacyCloud", "true").process(cloudClient); + } } @Override @@ -313,6 +319,29 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa assertSliceAndReplicaCount(collection1); assertUlogDir(collections); + + // restart all to test core saved state + + ChaosMonkey.stop(jettys); + ChaosMonkey.stop(controlJetty); + + assertTrue("Timeout waiting for all not live", ClusterStateUtil.waitForAllReplicasNotLive(cloudClient.getZkStateReader(), 45000)); + + ChaosMonkey.start(jettys); + ChaosMonkey.start(controlJetty); + + assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cloudClient.getZkStateReader(), collection1, 120000)); + + assertSliceAndReplicaCount(collection1); + + assertUlogDir(collections); + + assertSliceAndReplicaCount(collection1); + assertSingleReplicationAndShardSize(collection3, 5); + + // all docs should be queried + assertSingleReplicationAndShardSize(collection4, 5); + queryAndAssertResultSize(collection4, numDocs, 10000); } private void queryAndAssertResultSize(String collection, int expectedResultSize, int timeoutMS) From bb3d2e15f6379f349e1552a078b82bd86cf71023 Mon Sep 17 00:00:00 2001 From: markrmiller Date: Tue, 14 Mar 2017 06:13:34 -0400 Subject: [PATCH 04/20] SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs. --- solr/CHANGES.txt | 2 + .../admin/PropertiesRequestHandler.java | 27 +++++-- .../solr/handler/admin/SystemInfoHandler.java | 20 ++++- .../org/apache/solr/util/RedactionUtils.java | 51 +++++++++++++ .../admin/PropertiesRequestHandlerTest.java | 73 +++++++++++++++++++ 5 files changed, 165 insertions(+), 8 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/util/RedactionUtils.java create mode 100644 solr/core/src/test/org/apache/solr/handler/admin/PropertiesRequestHandlerTest.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 280f165a775..4c987dbfa30 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -181,6 +181,8 @@ New Features * SOLR-9045: Make RecoveryStrategy settings configurable. (Christine Poerschke) +* SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs. (Mano Kovacs via Mark Miller) + Bug Fixes ---------------------- diff --git a/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java index c16cded6e4c..57a7492ae75 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java @@ -17,12 +17,14 @@ package org.apache.solr.handler.admin; import java.io.IOException; +import java.util.Enumeration; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.handler.RequestHandlerBase; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.response.SolrQueryResponse; +import org.apache.solr.util.RedactionUtils; import static org.apache.solr.common.params.CommonParams.NAME; @@ -32,23 +34,36 @@ import static org.apache.solr.common.params.CommonParams.NAME; */ public class PropertiesRequestHandler extends RequestHandlerBase { + + public static final String REDACT_STRING = RedactionUtils.getRedactString(); + @Override public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException { - Object props = null; + NamedList props = new SimpleOrderedMap<>(); String name = req.getParams().get(NAME); if( name != null ) { - NamedList p = new SimpleOrderedMap<>(); - p.add( name, System.getProperty(name) ); - props = p; + String property = getSecuredPropertyValue(name); + props.add( name, property); } else { - props = System.getProperties(); + Enumeration enumeration = System.getProperties().propertyNames(); + while(enumeration.hasMoreElements()){ + name = (String) enumeration.nextElement(); + props.add(name, getSecuredPropertyValue(name)); + } } rsp.add( "system.properties", props ); rsp.setHttpCaching(false); } - + + private String getSecuredPropertyValue(String name) { + if(RedactionUtils.isSystemPropertySensitive(name)){ + return REDACT_STRING; + } + return System.getProperty(name); + } + //////////////////////// SolrInfoMBeans methods ////////////////////// @Override diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java index d031d69f685..94fb0553768 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java @@ -36,6 +36,8 @@ import java.text.DecimalFormat; import java.text.DecimalFormatSymbols; import java.util.Arrays; import java.util.Date; +import java.util.LinkedList; +import java.util.List; import java.util.Locale; import org.apache.commons.io.IOUtils; @@ -50,7 +52,7 @@ import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.schema.IndexSchema; import org.apache.solr.util.RTimer; - +import org.apache.solr.util.RedactionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,6 +68,8 @@ public class SystemInfoHandler extends RequestHandlerBase { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + public static String REDACT_STRING = RedactionUtils.getRedactString(); + /** *

* Undocumented expert level system property to prevent doing a reverse lookup of our hostname. @@ -373,7 +377,7 @@ public class SystemInfoHandler extends RequestHandlerBase // the input arguments passed to the Java virtual machine // which does not include the arguments to the main method. - jmx.add( "commandLineArgs", mx.getInputArguments()); + jmx.add( "commandLineArgs", getInputArgumentsRedacted(mx)); jmx.add( "startTime", new Date(mx.getStartTime())); jmx.add( "upTimeMS", mx.getUptime() ); @@ -436,6 +440,18 @@ public class SystemInfoHandler extends RequestHandlerBase return newSizeAndUnits; } + + private static List getInputArgumentsRedacted(RuntimeMXBean mx) { + List list = new LinkedList<>(); + for (String arg : mx.getInputArguments()) { + if (arg.startsWith("-D") && arg.contains("=") && RedactionUtils.isSystemPropertySensitive(arg.substring(2, arg.indexOf("=")))) { + list.add(String.format("%s=%s", arg.substring(0, arg.indexOf("=")), REDACT_STRING)); + } else { + list.add(arg); + } + } + return list; + } } diff --git a/solr/core/src/java/org/apache/solr/util/RedactionUtils.java b/solr/core/src/java/org/apache/solr/util/RedactionUtils.java new file mode 100644 index 00000000000..afa2abf952a --- /dev/null +++ b/solr/core/src/java/org/apache/solr/util/RedactionUtils.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.solr.util; + +import java.util.regex.Pattern; + +public class RedactionUtils { + public static final String SOLR_REDACTION_SYSTEM_PATTERN_PROP = "solr.redaction.system.pattern"; + private static Pattern pattern = Pattern.compile(System.getProperty(SOLR_REDACTION_SYSTEM_PATTERN_PROP, ".*password.*"), Pattern.CASE_INSENSITIVE); + private static final String REDACT_STRING = "--REDACTED--"; + + private static boolean redactSystemProperty = Boolean.parseBoolean(System.getProperty("solr.redaction.system.enabled", "true")); + + /** + * Returns if the given system property should be redacted. + * + * @param name The system property that is being checked. + * @return true if property should be redacted. + */ + static public boolean isSystemPropertySensitive(String name) { + return redactSystemProperty && pattern.matcher(name).matches(); + } + + /** + * @return redaction string to be used instead of the value. + */ + static public String getRedactString() { + return REDACT_STRING; + } + + public static void setRedactSystemProperty(boolean redactSystemProperty) { + RedactionUtils.redactSystemProperty = redactSystemProperty; + } + + +} diff --git a/solr/core/src/test/org/apache/solr/handler/admin/PropertiesRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/PropertiesRequestHandlerTest.java new file mode 100644 index 00000000000..1a959a45116 --- /dev/null +++ b/solr/core/src/test/org/apache/solr/handler/admin/PropertiesRequestHandlerTest.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.solr.handler.admin; + +import java.io.StringReader; + +import org.apache.solr.SolrTestCaseJ4; +import org.apache.solr.client.solrj.impl.XMLResponseParser; +import org.apache.solr.common.params.CommonParams; +import org.apache.solr.common.util.NamedList; +import org.apache.solr.util.RedactionUtils; +import org.junit.BeforeClass; +import org.junit.Test; + + +public class PropertiesRequestHandlerTest extends SolrTestCaseJ4 { + + public static final String PASSWORD = "secret123"; + public static final String REDACT_STRING = RedactionUtils.getRedactString(); + + + @BeforeClass + public static void beforeClass() throws Exception { + initCore("solrconfig.xml", "schema.xml"); + } + + @Test + public void testRedaction() throws Exception { + RedactionUtils.setRedactSystemProperty(true); + for(String propName: new String[]{"some.password", "javax.net.ssl.trustStorePassword"}){ + System.setProperty(propName, PASSWORD); + NamedList>> properties = readProperties(); + + assertEquals("Failed to redact "+propName, REDACT_STRING, properties.get(propName)); + } + } + + @Test + public void testDisabledRedaction() throws Exception { + RedactionUtils.setRedactSystemProperty(false); + for(String propName: new String[]{"some.password", "javax.net.ssl.trustStorePassword"}){ + System.setProperty(propName, PASSWORD); + NamedList>> properties = readProperties(); + + assertEquals("Failed to *not* redact "+propName, PASSWORD, properties.get(propName)); + } + } + + private NamedList>> readProperties() throws Exception { + String xml = h.query(req( + CommonParams.QT, "/admin/properties", + CommonParams.WT, "xml" + )); + + XMLResponseParser parser = new XMLResponseParser(); + return (NamedList>>) + parser.processResponse(new StringReader(xml)).get("system.properties"); + } +} \ No newline at end of file From 6083f370a15329dd476d817039d28af3d43ba213 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Tue, 14 Mar 2017 10:46:25 +0000 Subject: [PATCH 05/20] SOLR-10076: 'String.format(Locale.ROOT,...' instead of (forbidden API) 'String.format(...)' --- .../java/org/apache/solr/handler/admin/SystemInfoHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java index 94fb0553768..fc1679ff896 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java @@ -445,7 +445,7 @@ public class SystemInfoHandler extends RequestHandlerBase List list = new LinkedList<>(); for (String arg : mx.getInputArguments()) { if (arg.startsWith("-D") && arg.contains("=") && RedactionUtils.isSystemPropertySensitive(arg.substring(2, arg.indexOf("=")))) { - list.add(String.format("%s=%s", arg.substring(0, arg.indexOf("=")), REDACT_STRING)); + list.add(String.format(Locale.ROOT, "%s=%s", arg.substring(0, arg.indexOf("=")), REDACT_STRING)); } else { list.add(arg); } From 52b3bc2f6d16515f7f1e85a3a2e499ac68391427 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 14 Mar 2017 11:17:56 +0100 Subject: [PATCH 06/20] SOLR-10247: Support non-numeric metrics and a "compact" format of /admin/metrics. --- solr/CHANGES.txt | 2 + .../org/apache/solr/core/CoreContainer.java | 18 ++--- .../java/org/apache/solr/core/SolrCore.java | 36 ++++++--- .../solr/handler/admin/MetricsHandler.java | 6 +- .../solr/metrics/SolrMetricManager.java | 5 +- .../metrics/reporters/solr/SolrReporter.java | 2 +- .../solr/update/DirectUpdateHandler2.java | 22 ++---- .../apache/solr/update/SolrIndexWriter.java | 25 ++---- .../org/apache/solr/update/UpdateLog.java | 14 +--- ...tedPoolingHttpClientConnectionManager.java | 38 ++-------- .../apache/solr/util/stats/MetricUtils.java | 76 ++++++++++++------- .../handler/admin/MetricsHandlerTest.java | 22 +++++- .../solr/update/DirectUpdateHandlerTest.java | 18 ++--- .../solr/update/SolrIndexMetricsTest.java | 5 +- .../solr/util/stats/MetricUtilsTest.java | 53 ++++++++++++- 15 files changed, 195 insertions(+), 147 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 4c987dbfa30..94e52572066 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -339,6 +339,8 @@ Other Changes * SOLR-8876: change morphline test config files to work around 'importCommands' bug when using java9 (hossman) +* SOLR-10247: Support non-numeric metrics and a "compact" format of /admin/metrics output. (ab) + ================== 6.4.2 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java index 0de671e839d..2da48b828e3 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -36,7 +36,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import com.codahale.metrics.Gauge; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import org.apache.http.auth.AuthSchemeProvider; @@ -532,16 +531,13 @@ public class CoreContainer { containerProperties.putAll(cfg.getSolrProperties()); // initialize gauges for reporting the number of cores - Gauge loadedCores = () -> solrCores.getCores().size(); - Gauge lazyCores = () -> solrCores.getCoreNames().size() - solrCores.getCores().size(); - Gauge unloadedCores = () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size(); - - metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node), - loadedCores, true, "loaded", SolrInfoMBean.Category.CONTAINER.toString(), "cores"); - metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node), - lazyCores, true, "lazy",SolrInfoMBean.Category.CONTAINER.toString(), "cores"); - metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node), - unloadedCores, true, "unloaded",SolrInfoMBean.Category.CONTAINER.toString(), "cores"); + String registryName = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node); + metricManager.registerGauge(registryName, () -> solrCores.getCores().size(), + true, "loaded", SolrInfoMBean.Category.CONTAINER.toString(), "cores"); + metricManager.registerGauge(registryName, () -> solrCores.getCoreNames().size() - solrCores.getCores().size(), + true, "lazy",SolrInfoMBean.Category.CONTAINER.toString(), "cores"); + metricManager.registerGauge(registryName, () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size(), + true, "unloaded",SolrInfoMBean.Category.CONTAINER.toString(), "cores"); if (isZooKeeperAware()) { metricManager.loadClusterReporters(cfg.getMetricReporterPlugins(), this); diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java index 70203d4f2bd..bc41b1900f0 100644 --- a/solr/core/src/java/org/apache/solr/core/SolrCore.java +++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java @@ -169,7 +169,7 @@ import static org.apache.solr.common.params.CommonParams.PATH; /** * */ -public final class SolrCore implements SolrInfoMBean, Closeable { +public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closeable { public static final String version="1.0"; @@ -214,11 +214,11 @@ public final class SolrCore implements SolrInfoMBean, Closeable { private final ReentrantLock ruleExpiryLock; private final ReentrantLock snapshotDelLock; // A lock instance to guard against concurrent deletions. - private final Timer newSearcherTimer; - private final Timer newSearcherWarmupTimer; - private final Counter newSearcherCounter; - private final Counter newSearcherMaxReachedCounter; - private final Counter newSearcherOtherErrorsCounter; + private Timer newSearcherTimer; + private Timer newSearcherWarmupTimer; + private Counter newSearcherCounter; + private Counter newSearcherMaxReachedCounter; + private Counter newSearcherOtherErrorsCounter; public Date getStartTimeStamp() { return startTime; } @@ -901,11 +901,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable { SolrMetricManager metricManager = this.coreDescriptor.getCoreContainer().getMetricManager(); // initialize searcher-related metrics - newSearcherCounter = metricManager.counter(coreMetricManager.getRegistryName(), "new", Category.SEARCHER.toString()); - newSearcherTimer = metricManager.timer(coreMetricManager.getRegistryName(), "time", Category.SEARCHER.toString(), "new"); - newSearcherWarmupTimer = metricManager.timer(coreMetricManager.getRegistryName(), "warmup", Category.SEARCHER.toString(), "new"); - newSearcherMaxReachedCounter = metricManager.counter(coreMetricManager.getRegistryName(), "maxReached", Category.SEARCHER.toString(), "new"); - newSearcherOtherErrorsCounter = metricManager.counter(coreMetricManager.getRegistryName(), "errors", Category.SEARCHER.toString(), "new"); + initializeMetrics(metricManager, coreMetricManager.getRegistryName(), null); // Initialize JMX this.infoRegistry = initInfoRegistry(name, config); @@ -1125,6 +1121,24 @@ public final class SolrCore implements SolrInfoMBean, Closeable { return coreMetricManager; } + @Override + public void initializeMetrics(SolrMetricManager manager, String registry, String scope) { + newSearcherCounter = manager.counter(registry, "new", Category.SEARCHER.toString()); + newSearcherTimer = manager.timer(registry, "time", Category.SEARCHER.toString(), "new"); + newSearcherWarmupTimer = manager.timer(registry, "warmup", Category.SEARCHER.toString(), "new"); + newSearcherMaxReachedCounter = manager.counter(registry, "maxReached", Category.SEARCHER.toString(), "new"); + newSearcherOtherErrorsCounter = manager.counter(registry, "errors", Category.SEARCHER.toString(), "new"); + + manager.registerGauge(registry, () -> name == null ? "(null)" : name, true, "coreName", Category.CORE.toString()); + manager.registerGauge(registry, () -> startTime, true, "startTime", Category.CORE.toString()); + manager.registerGauge(registry, () -> getOpenCount(), true, "refCount", Category.CORE.toString()); + manager.registerGauge(registry, () -> resourceLoader.getInstancePath(), true, "instanceDir", Category.CORE.toString()); + manager.registerGauge(registry, () -> getIndexDir(), true, "indexDir", Category.CORE.toString()); + manager.registerGauge(registry, () -> getIndexSize(), true, "sizeInBytes", Category.INDEX.toString()); + manager.registerGauge(registry, () -> NumberUtils.readableSize(getIndexSize()), true, "size", Category.INDEX.toString()); + manager.registerGauge(registry, () -> coreDescriptor.getCoreContainer().getCoreNames(this), true, "aliases", Category.CORE.toString()); + } + private Map initInfoRegistry(String name, SolrConfig config) { if (config.jmxConfig.enabled) { return new JmxMonitoredMap(name, coreMetricManager.getRegistryName(), String.valueOf(this.hashCode()), config.jmxConfig); diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java index b53c818ec7e..25f317c00d7 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java @@ -50,6 +50,8 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName final CoreContainer container; final SolrMetricManager metricManager; + public static final String COMPACT_PARAM = "compact"; + public MetricsHandler() { this.container = null; this.metricManager = null; @@ -71,6 +73,7 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Core container instance not initialized"); } + boolean compact = req.getParams().getBool(COMPACT_PARAM, false); MetricFilter mustMatchFilter = parseMustMatchFilter(req); List metricTypes = parseMetricTypes(req); List metricFilters = metricTypes.stream().map(MetricType::asMetricFilter).collect(Collectors.toList()); @@ -79,7 +82,8 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName NamedList response = new NamedList(); for (String registryName : requestedRegistries) { MetricRegistry registry = metricManager.registry(registryName); - response.add(registryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter, false, false, null)); + response.add(registryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter, false, + false, compact, null)); } rsp.getValues().add("metrics", response); } diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java index 3a4c3fed770..f4abee0cc92 100644 --- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java +++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java @@ -38,6 +38,7 @@ import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; import com.codahale.metrics.Histogram; import com.codahale.metrics.Meter; import com.codahale.metrics.Metric; @@ -520,7 +521,9 @@ public class SolrMetricManager { } } - + public void registerGauge(String registry, Gauge gauge, boolean force, String metricName, String... metricPath) { + register(registry, gauge, force, metricName, metricPath); + } /** * This method creates a hierarchical name with arbitrary levels of hierarchy diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java index e9b8c3dc2c7..0bca68f52df 100644 --- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java +++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java @@ -361,7 +361,7 @@ public class SolrReporter extends ScheduledReporter { } final String effectiveGroup = group; MetricUtils.toSolrInputDocuments(metricManager.registry(registryName), Collections.singletonList(report.filter), MetricFilter.ALL, - skipHistograms, skipAggregateValues, metadata, doc -> { + skipHistograms, skipAggregateValues, false, metadata, doc -> { doc.setField(REGISTRY_ID, registryName); doc.setField(GROUP_ID, effectiveGroup); if (effectiveLabel != null) { diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java index abb5512c841..fdc9d2223ec 100644 --- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java +++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java @@ -26,7 +26,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.atomic.LongAdder; -import com.codahale.metrics.Gauge; import com.codahale.metrics.Meter; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -164,25 +163,18 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState @Override public void initializeMetrics(SolrMetricManager manager, String registry, String scope) { commitCommands = manager.meter(registry, "commits", getCategory().toString(), scope); - Gauge autoCommits = () -> commitTracker.getCommitCount(); - manager.register(registry, autoCommits, true, "autoCommits", getCategory().toString(), scope); - Gauge softAutoCommits = () -> softCommitTracker.getCommitCount(); - manager.register(registry, softAutoCommits, true, "softAutoCommits", getCategory().toString(), scope); + manager.registerGauge(registry, () -> commitTracker.getCommitCount(), true, "autoCommits", getCategory().toString(), scope); + manager.registerGauge(registry, () -> softCommitTracker.getCommitCount(), true, "softAutoCommits", getCategory().toString(), scope); optimizeCommands = manager.meter(registry, "optimizes", getCategory().toString(), scope); rollbackCommands = manager.meter(registry, "rollbacks", getCategory().toString(), scope); splitCommands = manager.meter(registry, "splits", getCategory().toString(), scope); mergeIndexesCommands = manager.meter(registry, "merges", getCategory().toString(), scope); expungeDeleteCommands = manager.meter(registry, "expungeDeletes", getCategory().toString(), scope); - Gauge docsPending = () -> numDocsPending.longValue(); - manager.register(registry, docsPending, true, "docsPending", getCategory().toString(), scope); - Gauge adds = () -> addCommands.longValue(); - manager.register(registry, adds, true, "adds", getCategory().toString(), scope); - Gauge deletesById = () -> deleteByIdCommands.longValue(); - manager.register(registry, deletesById, true, "deletesById", getCategory().toString(), scope); - Gauge deletesByQuery = () -> deleteByQueryCommands.longValue(); - manager.register(registry, deletesByQuery, true, "deletesByQuery", getCategory().toString(), scope); - Gauge errors = () -> numErrors.longValue(); - manager.register(registry, errors, true, "errors", getCategory().toString(), scope); + manager.registerGauge(registry, () -> numDocsPending.longValue(), true, "docsPending", getCategory().toString(), scope); + manager.registerGauge(registry, () -> addCommands.longValue(), true, "adds", getCategory().toString(), scope); + manager.registerGauge(registry, () -> deleteByIdCommands.longValue(), true, "deletesById", getCategory().toString(), scope); + manager.registerGauge(registry, () -> deleteByQueryCommands.longValue(), true, "deletesByQuery", getCategory().toString(), scope); + manager.registerGauge(registry, () -> numErrors.longValue(), true, "errors", getCategory().toString(), scope); addCommandsCumulative = manager.meter(registry, "cumulativeAdds", getCategory().toString(), scope); deleteByIdCommandsCumulative = manager.meter(registry, "cumulativeDeletesById", getCategory().toString(), scope); diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java b/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java index 6a264f851dd..ed856040ec1 100644 --- a/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java +++ b/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import com.codahale.metrics.Counter; -import com.codahale.metrics.Gauge; import com.codahale.metrics.Meter; import com.codahale.metrics.Timer; import org.apache.lucene.codecs.Codec; @@ -81,17 +80,11 @@ public class SolrIndexWriter extends IndexWriter { private boolean mergeTotals = false; private boolean mergeDetails = false; private final AtomicInteger runningMajorMerges = new AtomicInteger(); - private Gauge runningMajorMergesGauge; private final AtomicInteger runningMinorMerges = new AtomicInteger(); - private Gauge runningMinorMergesGauge; private final AtomicInteger runningMajorMergesSegments = new AtomicInteger(); - private Gauge runningMajorMergesSegmentsGauge; private final AtomicInteger runningMinorMergesSegments = new AtomicInteger(); - private Gauge runningMinorMergesSegmentsGauge; private final AtomicLong runningMajorMergesDocs = new AtomicLong(); - private Gauge runningMajorMergesDocsGauge; private final AtomicLong runningMinorMergesDocs = new AtomicLong(); - private Gauge runningMinorMergesDocsGauge; public static SolrIndexWriter create(SolrCore core, String name, String path, DirectoryFactory directoryFactory, boolean create, IndexSchema schema, SolrIndexConfig config, IndexDeletionPolicy delPolicy, Codec codec) throws IOException { @@ -165,18 +158,12 @@ public class SolrIndexWriter extends IndexWriter { minorMerge = metricManager.timer(registry, "minor", SolrInfoMBean.Category.INDEX.toString(), "merge"); majorMerge = metricManager.timer(registry, "major", SolrInfoMBean.Category.INDEX.toString(), "merge"); mergeErrors = metricManager.counter(registry, "errors", SolrInfoMBean.Category.INDEX.toString(), "merge"); - runningMajorMergesGauge = () -> runningMajorMerges.get(); - runningMinorMergesGauge = () -> runningMinorMerges.get(); - runningMajorMergesDocsGauge = () -> runningMajorMergesDocs.get(); - runningMinorMergesDocsGauge = () -> runningMinorMergesDocs.get(); - runningMajorMergesSegmentsGauge = () -> runningMajorMergesSegments.get(); - runningMinorMergesSegmentsGauge = () -> runningMinorMergesSegments.get(); - metricManager.register(registry, runningMajorMergesGauge, true, "running", SolrInfoMBean.Category.INDEX.toString(), "merge", "major"); - metricManager.register(registry, runningMinorMergesGauge, true, "running", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor"); - metricManager.register(registry, runningMajorMergesDocsGauge, true, "running.docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "major"); - metricManager.register(registry, runningMinorMergesDocsGauge, true, "running.docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor"); - metricManager.register(registry, runningMajorMergesSegmentsGauge, true, "running.segments", SolrInfoMBean.Category.INDEX.toString(), "merge", "major"); - metricManager.register(registry, runningMinorMergesSegmentsGauge, true, "running.segments", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor"); + metricManager.registerGauge(registry, () -> runningMajorMerges.get(), true, "running", SolrInfoMBean.Category.INDEX.toString(), "merge", "major"); + metricManager.registerGauge(registry, () -> runningMinorMerges.get(), true, "running", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor"); + metricManager.registerGauge(registry, () -> runningMajorMergesDocs.get(), true, "running.docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "major"); + metricManager.registerGauge(registry, () -> runningMinorMergesDocs.get(), true, "running.docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor"); + metricManager.registerGauge(registry, () -> runningMajorMergesSegments.get(), true, "running.segments", SolrInfoMBean.Category.INDEX.toString(), "merge", "major"); + metricManager.registerGauge(registry, () -> runningMinorMergesSegments.get(), true, "running.segments", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor"); flushMeter = metricManager.meter(registry, "flush", SolrInfoMBean.Category.INDEX.toString()); } } diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java index 6a5f407daa1..bd0adbb4252 100644 --- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java +++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java @@ -234,9 +234,6 @@ public static final int VERSION_IDX = 1; // metrics protected Gauge bufferedOpsGauge; - protected Gauge replayLogsCountGauge; - protected Gauge replayBytesGauge; - protected Gauge stateGauge; protected Meter applyingBufferedOpsMeter; protected Meter replayOpsMeter; @@ -424,16 +421,13 @@ public static final int VERSION_IDX = 1; return 0; } }; - replayLogsCountGauge = () -> logs.size(); - replayBytesGauge = () -> getTotalLogsSize(); - manager.register(registry, bufferedOpsGauge, true, "ops", scope, "buffered"); - manager.register(registry, replayLogsCountGauge, true, "logs", scope, "replay", "remaining"); - manager.register(registry, replayBytesGauge, true, "bytes", scope, "replay", "remaining"); + manager.registerGauge(registry, bufferedOpsGauge, true, "ops", scope, "buffered"); + manager.registerGauge(registry, () -> logs.size(), true, "logs", scope, "replay", "remaining"); + manager.registerGauge(registry, () -> getTotalLogsSize(), true, "bytes", scope, "replay", "remaining"); applyingBufferedOpsMeter = manager.meter(registry, "ops", scope, "applyingBuffered"); replayOpsMeter = manager.meter(registry, "ops", scope, "replay"); - stateGauge = () -> state.getValue(); - manager.register(registry, stateGauge, true, "state", scope); + manager.registerGauge(registry, () -> state.getValue(), true, "state", scope); } /** diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java index 6c992fa36c5..7bcabf8c255 100644 --- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java +++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedPoolingHttpClientConnectionManager.java @@ -17,8 +17,6 @@ package org.apache.solr.util.stats; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.MetricRegistry; import org.apache.http.config.Registry; import org.apache.http.conn.socket.ConnectionSocketFactory; import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; @@ -31,42 +29,16 @@ import org.apache.solr.metrics.SolrMetricProducer; */ public class InstrumentedPoolingHttpClientConnectionManager extends PoolingHttpClientConnectionManager implements SolrMetricProducer { - protected MetricRegistry metricsRegistry; - public InstrumentedPoolingHttpClientConnectionManager(Registry socketFactoryRegistry) { super(socketFactoryRegistry); } - public MetricRegistry getMetricsRegistry() { - return metricsRegistry; - } - - public void setMetricsRegistry(MetricRegistry metricRegistry) { - this.metricsRegistry = metricRegistry; - } - @Override public void initializeMetrics(SolrMetricManager manager, String registry, String scope) { - this.metricsRegistry = manager.registry(registry); - metricsRegistry.register(SolrMetricManager.mkName("availableConnections", scope), - (Gauge) () -> { - // this acquires a lock on the connection pool; remove if contention sucks - return getTotalStats().getAvailable(); - }); - metricsRegistry.register(SolrMetricManager.mkName("leasedConnections", scope), - (Gauge) () -> { - // this acquires a lock on the connection pool; remove if contention sucks - return getTotalStats().getLeased(); - }); - metricsRegistry.register(SolrMetricManager.mkName("maxConnections", scope), - (Gauge) () -> { - // this acquires a lock on the connection pool; remove if contention sucks - return getTotalStats().getMax(); - }); - metricsRegistry.register(SolrMetricManager.mkName("pendingConnections", scope), - (Gauge) () -> { - // this acquires a lock on the connection pool; remove if contention sucks - return getTotalStats().getPending(); - }); + manager.registerGauge(registry, () -> getTotalStats().getAvailable(), true, SolrMetricManager.mkName("availableConnections", scope)); + // this acquires a lock on the connection pool; remove if contention sucks + manager.registerGauge(registry, () -> getTotalStats().getLeased(), true, SolrMetricManager.mkName("leasedConnections", scope)); + manager.registerGauge(registry, () -> getTotalStats().getMax(), true, SolrMetricManager.mkName("maxConnections", scope)); + manager.registerGauge(registry, () -> getTotalStats().getPending(), true, SolrMetricManager.mkName("pendingConnections", scope)); } } diff --git a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java index 70fd4674c9e..ba1284e3b72 100644 --- a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java +++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java @@ -47,6 +47,7 @@ import org.apache.solr.metrics.AggregateMetric; public class MetricUtils { public static final String METRIC_NAME = "metric"; + public static final String VALUE = "value"; public static final String VALUES = "values"; static final String MS = "_ms"; @@ -110,16 +111,17 @@ public class MetricUtils { * @param mustMatchFilter a {@link MetricFilter}. * A metric must match this filter to be included in the output. * @param skipHistograms discard any {@link Histogram}-s and histogram parts of {@link Timer}-s. + * @param compact use compact representation for counters and gauges. * @param metadata optional metadata. If not null and not empty then this map will be added under a * {@code _metadata_} key. * @return a {@link NamedList} */ public static NamedList toNamedList(MetricRegistry registry, List shouldMatchFilters, MetricFilter mustMatchFilter, boolean skipHistograms, - boolean skipAggregateValues, + boolean skipAggregateValues, boolean compact, Map metadata) { NamedList result = new SimpleOrderedMap(); - toNamedMaps(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, skipAggregateValues, (k, v) -> { + toMaps(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, skipAggregateValues, compact, (k, v) -> { result.add(k, v); }); if (metadata != null && !metadata.isEmpty()) { @@ -140,17 +142,18 @@ public class MetricUtils { * @param mustMatchFilter a {@link MetricFilter}. * A metric must match this filter to be included in the output. * @param skipHistograms discard any {@link Histogram}-s and histogram parts of {@link Timer}-s. + * @param compact use compact representation for counters and gauges. * @param metadata optional metadata. If not null and not empty then this map will be added under a * {@code _metadata_} key. * @return a list of {@link SolrInputDocument}-s */ public static List toSolrInputDocuments(MetricRegistry registry, List shouldMatchFilters, MetricFilter mustMatchFilter, boolean skipHistograms, - boolean skipAggregateValues, + boolean skipAggregateValues, boolean compact, Map metadata) { List result = new LinkedList<>(); toSolrInputDocuments(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, - skipAggregateValues, metadata, doc -> { + skipAggregateValues, compact, metadata, doc -> { result.add(doc); }); return result; @@ -158,10 +161,10 @@ public class MetricUtils { public static void toSolrInputDocuments(MetricRegistry registry, List shouldMatchFilters, MetricFilter mustMatchFilter, boolean skipHistograms, - boolean skipAggregateValues, + boolean skipAggregateValues, boolean compact, Map metadata, Consumer consumer) { boolean addMetadata = metadata != null && !metadata.isEmpty(); - toNamedMaps(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, skipAggregateValues, (k, v) -> { + toMaps(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, skipAggregateValues, compact, (k, v) -> { SolrInputDocument doc = new SolrInputDocument(); doc.setField(METRIC_NAME, k); toSolrInputDocument(null, doc, v); @@ -172,10 +175,16 @@ public class MetricUtils { }); } - public static void toSolrInputDocument(String prefix, SolrInputDocument doc, Map map) { + public static void toSolrInputDocument(String prefix, SolrInputDocument doc, Object o) { + if (!(o instanceof Map)) { + String key = prefix != null ? prefix : VALUE; + doc.addField(key, o); + return; + } + Map map = (Map)o; for (Map.Entry entry : map.entrySet()) { if (entry.getValue() instanceof Map) { // flatten recursively - toSolrInputDocument(entry.getKey(), doc, (Map)entry.getValue()); + toSolrInputDocument(entry.getKey(), doc, entry.getValue()); } else { String key = prefix != null ? prefix + "." + entry.getKey() : entry.getKey(); doc.addField(key, entry.getValue()); @@ -183,9 +192,10 @@ public class MetricUtils { } } - public static void toNamedMaps(MetricRegistry registry, List shouldMatchFilters, - MetricFilter mustMatchFilter, boolean skipHistograms, boolean skipAggregateValues, - BiConsumer> consumer) { + public static void toMaps(MetricRegistry registry, List shouldMatchFilters, + MetricFilter mustMatchFilter, boolean skipHistograms, boolean skipAggregateValues, + boolean compact, + BiConsumer consumer) { Map metrics = registry.getMetrics(); SortedSet names = registry.getNames(); names.stream() @@ -195,28 +205,28 @@ public class MetricUtils { Metric metric = metrics.get(n); if (metric instanceof Counter) { Counter counter = (Counter) metric; - consumer.accept(n, counterToMap(counter)); + consumer.accept(n, convertCounter(counter, compact)); } else if (metric instanceof Gauge) { Gauge gauge = (Gauge) metric; - consumer.accept(n, gaugeToMap(gauge)); + consumer.accept(n, convertGauge(gauge, compact)); } else if (metric instanceof Meter) { Meter meter = (Meter) metric; - consumer.accept(n, meterToMap(meter)); + consumer.accept(n, convertMeter(meter)); } else if (metric instanceof Timer) { Timer timer = (Timer) metric; - consumer.accept(n, timerToMap(timer, skipHistograms)); + consumer.accept(n, convertTimer(timer, skipHistograms)); } else if (metric instanceof Histogram) { if (!skipHistograms) { Histogram histogram = (Histogram) metric; - consumer.accept(n, histogramToMap(histogram)); + consumer.accept(n, convertHistogram(histogram)); } } else if (metric instanceof AggregateMetric) { - consumer.accept(n, aggregateMetricToMap((AggregateMetric)metric, skipAggregateValues)); + consumer.accept(n, convertAggregateMetric((AggregateMetric)metric, skipAggregateValues)); } }); } - static Map aggregateMetricToMap(AggregateMetric metric, boolean skipAggregateValues) { + static Map convertAggregateMetric(AggregateMetric metric, boolean skipAggregateValues) { Map response = new LinkedHashMap<>(); response.put("count", metric.size()); response.put(MAX, metric.getMax()); @@ -237,7 +247,7 @@ public class MetricUtils { return response; } - static Map histogramToMap(Histogram histogram) { + static Map convertHistogram(Histogram histogram) { Map response = new LinkedHashMap<>(); Snapshot snapshot = histogram.getSnapshot(); response.put("count", histogram.getCount()); @@ -268,7 +278,7 @@ public class MetricUtils { response.put((ms ? P999_MS: P999), nsToMs(ms, snapshot.get999thPercentile())); } - static Map timerToMap(Timer timer, boolean skipHistograms) { + static Map convertTimer(Timer timer, boolean skipHistograms) { Map response = new LinkedHashMap<>(); response.put("count", timer.getCount()); response.put("meanRate", timer.getMeanRate()); @@ -282,7 +292,7 @@ public class MetricUtils { return response; } - static Map meterToMap(Meter meter) { + static Map convertMeter(Meter meter) { Map response = new LinkedHashMap<>(); response.put("count", meter.getCount()); response.put("meanRate", meter.getMeanRate()); @@ -292,16 +302,24 @@ public class MetricUtils { return response; } - static Map gaugeToMap(Gauge gauge) { - Map response = new LinkedHashMap<>(); - response.put("value", gauge.getValue()); - return response; + static Object convertGauge(Gauge gauge, boolean compact) { + if (compact) { + return gauge.getValue(); + } else { + Map response = new LinkedHashMap<>(); + response.put("value", gauge.getValue()); + return response; + } } - static Map counterToMap(Counter counter) { - Map response = new LinkedHashMap<>(); - response.put("count", counter.getCount()); - return response; + static Object convertCounter(Counter counter, boolean compact) { + if (compact) { + return counter.getCount(); + } else { + Map response = new LinkedHashMap<>(); + response.put("count", counter.getCount()); + return response; + } } /** diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java index 2fd7e9ca7b5..81e14d944a5 100644 --- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java @@ -49,9 +49,11 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 { assertNotNull(values.get("solr.node")); NamedList nl = (NamedList) values.get("solr.core.collection1"); assertNotNull(nl); - assertNotNull(nl.get("SEARCHER.new.errors")); // counter type - assertNotNull(((Map) nl.get("SEARCHER.new.errors")).get("count")); + Object o = nl.get("SEARCHER.new.errors"); + assertNotNull(o); // counter type + assertTrue(o instanceof Map); // response wasn't serialized so we get here whatever MetricUtils produced instead of NamedList + assertNotNull(((Map) o).get("count")); assertEquals(0L, ((Map) nl.get("SEARCHER.new.errors")).get("count")); nl = (NamedList) values.get("solr.node"); assertNotNull(nl.get("CONTAINER.cores.loaded")); // int gauge @@ -149,4 +151,20 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 { assertEquals(1, values.size()); assertEquals(0, ((NamedList)values.get("solr.node")).size()); } + + @Test + public void testCompact() throws Exception { + MetricsHandler handler = new MetricsHandler(h.getCoreContainer()); + + SolrQueryResponse resp = new SolrQueryResponse(); + handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", MetricsHandler.COMPACT_PARAM, "true"), resp); + NamedList values = resp.getValues(); + assertNotNull(values.get("metrics")); + values = (NamedList) values.get("metrics"); + NamedList nl = (NamedList) values.get("solr.core.collection1"); + assertNotNull(nl); + Object o = nl.get("SEARCHER.new.errors"); + assertNotNull(o); // counter type + assertTrue(o instanceof Number); + } } diff --git a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java index 281635435e4..462241a3902 100644 --- a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java @@ -119,7 +119,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { String delsQName = PREFIX + "deletesByQuery"; String cumulativeDelsQName = PREFIX + "cumulativeDeletesByQuery"; long commits = ((Meter) metrics.get(commitsName)).getCount(); - long adds = ((Gauge) metrics.get(addsName)).getValue(); + long adds = ((Gauge) metrics.get(addsName)).getValue().longValue(); long cumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount(); long cumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount(); long cumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount(); @@ -137,7 +137,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { assertQ(req("q","id:5"), "//*[@numFound='0']"); assertQ(req("q","id:6"), "//*[@numFound='0']"); - long newAdds = ((Gauge) metrics.get(addsName)).getValue(); + long newAdds = ((Gauge) metrics.get(addsName)).getValue().longValue(); long newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount(); assertEquals("new adds", 2, newAdds - adds); assertEquals("new cumulative adds", 2, newCumulativeAdds - cumulativeAdds); @@ -147,7 +147,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { long newCommits = ((Meter) metrics.get(commitsName)).getCount(); assertEquals("new commits", 1, newCommits - commits); - newAdds = ((Gauge) metrics.get(addsName)).getValue(); + newAdds = ((Gauge) metrics.get(addsName)).getValue().longValue(); newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount(); // adds should be reset to 0 after commit assertEquals("new adds after commit", 0, newAdds); @@ -161,7 +161,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { // now delete one assertU(delI("5")); - long newDelsI = ((Gauge) metrics.get(delsIName)).getValue(); + long newDelsI = ((Gauge) metrics.get(delsIName)).getValue().longValue(); long newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount(); assertEquals("new delsI", 1, newDelsI); assertEquals("new cumulative delsI", 1, newCumulativeDelsI - cumulativeDelsI); @@ -171,7 +171,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { assertU(commit()); // delsI should be reset to 0 after commit - newDelsI = ((Gauge) metrics.get(delsIName)).getValue(); + newDelsI = ((Gauge) metrics.get(delsIName)).getValue().longValue(); newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount(); assertEquals("new delsI after commit", 0, newDelsI); assertEquals("new cumulative delsI after commit", 1, newCumulativeDelsI - cumulativeDelsI); @@ -183,7 +183,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { // now delete all assertU(delQ("*:*")); - long newDelsQ = ((Gauge) metrics.get(delsQName)).getValue(); + long newDelsQ = ((Gauge) metrics.get(delsQName)).getValue().longValue(); long newCumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount(); assertEquals("new delsQ", 1, newDelsQ); assertEquals("new cumulative delsQ", 1, newCumulativeDelsQ - cumulativeDelsQ); @@ -193,7 +193,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { assertU(commit()); - newDelsQ = ((Gauge) metrics.get(delsQName)).getValue(); + newDelsQ = ((Gauge) metrics.get(delsQName)).getValue().longValue(); newCumulativeDelsQ = ((Meter) metrics.get(cumulativeDelsQName)).getCount(); assertEquals("new delsQ after commit", 0, newDelsQ); assertEquals("new cumulative delsQ after commit", 1, newCumulativeDelsQ - cumulativeDelsQ); @@ -204,11 +204,11 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { // verify final metrics newCommits = ((Meter) metrics.get(commitsName)).getCount(); assertEquals("new commits", 3, newCommits - commits); - newAdds = ((Gauge) metrics.get(addsName)).getValue(); + newAdds = ((Gauge) metrics.get(addsName)).getValue().longValue(); assertEquals("new adds", 0, newAdds); newCumulativeAdds = ((Meter) metrics.get(cumulativeAddsName)).getCount(); assertEquals("new cumulative adds", 2, newCumulativeAdds - cumulativeAdds); - newDelsI = ((Gauge) metrics.get(delsIName)).getValue(); + newDelsI = ((Gauge) metrics.get(delsIName)).getValue().longValue(); assertEquals("new delsI", 0, newDelsI); newCumulativeDelsI = ((Meter) metrics.get(cumulativeDelsIName)).getCount(); assertEquals("new cumulative delsI", 1, newCumulativeDelsI - cumulativeDelsI); diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java index c9935bb3830..9985937c7f0 100644 --- a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java +++ b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java @@ -67,7 +67,7 @@ public class SolrIndexMetricsTest extends SolrTestCaseJ4 { Map metrics = registry.getMetrics(); - assertEquals(10, metrics.entrySet().stream().filter(e -> e.getKey().startsWith("INDEX")).count()); + assertEquals(12, metrics.entrySet().stream().filter(e -> e.getKey().startsWith("INDEX")).count()); // check basic index meters Timer timer = (Timer)metrics.get("INDEX.merge.minor"); @@ -92,7 +92,8 @@ public class SolrIndexMetricsTest extends SolrTestCaseJ4 { assertNotNull(registry); Map metrics = registry.getMetrics(); - assertEquals(0, metrics.entrySet().stream().filter(e -> e.getKey().startsWith("INDEX")).count()); + // INDEX.size, INDEX.sizeInBytes + assertEquals(2, metrics.entrySet().stream().filter(e -> e.getKey().startsWith("INDEX")).count()); } @Test diff --git a/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java index 8717ad641b3..35caef8af5d 100644 --- a/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java +++ b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; import com.codahale.metrics.Histogram; import com.codahale.metrics.Meter; import com.codahale.metrics.MetricFilter; @@ -44,7 +45,7 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { timer.update(Math.abs(random().nextInt()) + 1, TimeUnit.NANOSECONDS); } // obtain timer metrics - NamedList lst = new NamedList(MetricUtils.timerToMap(timer, false)); + NamedList lst = new NamedList(MetricUtils.convertTimer(timer, false)); // check that expected metrics were obtained assertEquals(14, lst.size()); final Snapshot snapshot = timer.getSnapshot(); @@ -78,10 +79,15 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { am.set("foo", 10); am.set("bar", 1); am.set("bar", 2); - MetricUtils.toNamedMaps(registry, Collections.singletonList(MetricFilter.ALL), MetricFilter.ALL, - false, false, (k, v) -> { + Gauge gauge = () -> "foobar"; + registry.register("gauge", gauge); + MetricUtils.toMaps(registry, Collections.singletonList(MetricFilter.ALL), MetricFilter.ALL, + false, false, false, (k, o) -> { + Map v = (Map)o; if (k.startsWith("counter")) { assertEquals(1L, v.get("count")); + } else if (k.startsWith("gauge")) { + assertEquals("foobar", v.get("value")); } else if (k.startsWith("timer")) { assertEquals(1L, v.get("count")); assertTrue(((Number)v.get("min_ms")).intValue() > 100); @@ -102,6 +108,47 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { assertEquals(2, update.get("updateCount")); } }); + // test compact format + MetricUtils.toMaps(registry, Collections.singletonList(MetricFilter.ALL), MetricFilter.ALL, + false, false, true, (k, o) -> { + if (k.startsWith("counter")) { + assertTrue(o instanceof Long); + assertEquals(1L, o); + } else if (k.startsWith("gauge")) { + assertTrue(o instanceof String); + assertEquals("foobar", o); + } else if (k.startsWith("timer")) { + assertTrue(o instanceof Map); + Map v = (Map)o; + assertEquals(1L, v.get("count")); + assertTrue(((Number)v.get("min_ms")).intValue() > 100); + } else if (k.startsWith("meter")) { + assertTrue(o instanceof Map); + Map v = (Map)o; + assertEquals(1L, v.get("count")); + } else if (k.startsWith("histogram")) { + assertTrue(o instanceof Map); + Map v = (Map)o; + assertEquals(1L, v.get("count")); + } else if (k.startsWith("aggregate")) { + assertTrue(o instanceof Map); + Map v = (Map)o; + assertEquals(2, v.get("count")); + Map values = (Map)v.get("values"); + assertNotNull(values); + assertEquals(2, values.size()); + Map update = (Map)values.get("foo"); + assertEquals(10, update.get("value")); + assertEquals(1, update.get("updateCount")); + update = (Map)values.get("bar"); + assertEquals(2, update.get("value")); + assertEquals(2, update.get("updateCount")); + } else { + Map v = (Map)o; + assertEquals(1L, v.get("count")); + } + }); + } } From 6163956a1b303ca3690715f91ad737dab78b84ea Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Tue, 14 Mar 2017 13:05:55 +0000 Subject: [PATCH 07/20] SOLR-9045: exclude static methods from ConfigureRecoveryStrategyTest.testAlmostAllMethodsAreFinal --- .../test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java b/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java index 80032afe2a0..0a988f6e747 100644 --- a/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java +++ b/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java @@ -66,6 +66,7 @@ public class ConfigureRecoveryStrategyTest extends SolrTestCaseJ4 { public void testAlmostAllMethodsAreFinal() throws Exception { for (Method m : RecoveryStrategy.class.getDeclaredMethods()) { + if (Modifier.isStatic(m.getModifiers())) continue; final String methodName = m.getName(); if ("getReplicateLeaderUrl".equals(methodName)) { assertFalse(m.toString(), Modifier.isFinal(m.getModifiers())); From 875690302c9d3be8aac81865685910d407588e1e Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Tue, 14 Mar 2017 09:49:22 -0700 Subject: [PATCH 08/20] SOLR-10266: fix implicit test assumptions about doc order --- .../apache/solr/schema/TestPointFields.java | 161 ++++++++++-------- 1 file changed, 94 insertions(+), 67 deletions(-) diff --git a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java index 3c1f0b3e21a..36e8c1031f7 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java +++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java @@ -940,52 +940,59 @@ public class TestPointFields extends SolrTestCaseJ4 { } private void doTestIntPointFieldRangeQuery(String fieldName, String type, boolean testLong) throws Exception { - for (int i = 0; i < 10; i++) { + for (int i = 9; i >= 0; i--) { assertU(adoc("id", String.valueOf(i), fieldName, String.valueOf(i))); } assertU(commit()); - assertQ(req("q", fieldName + ":[0 TO 3]", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[0 TO 3]", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1']", "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='2']", "//result/doc[4]/" + type + "[@name='" + fieldName + "'][.='3']"); - assertQ(req("q", fieldName + ":{0 TO 3]", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":{0 TO 3]", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='1']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='2']", "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='3']"); - assertQ(req("q", fieldName + ":[0 TO 3}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[0 TO 3}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1']", "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='2']"); - assertQ(req("q", fieldName + ":{0 TO 3}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":{0 TO 3}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='2']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='1']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='2']"); - assertQ(req("q", fieldName + ":{0 TO *}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":{0 TO *}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='9']", + "0=count(//result/doc/" + type + "[@name='" + fieldName + "'][.='0'])", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='1']"); - assertQ(req("q", fieldName + ":{* TO 3}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":{* TO 3}", "fl", "id, " + fieldName, "sort", "id desc"), "//*[@numFound='3']", - "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0']"); + "0=count(//result/doc/" + type + "[@name='" + fieldName + "'][.='3'])", + "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='2']", + "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1']", + "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='0']"); - assertQ(req("q", fieldName + ":[* TO 3}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[* TO 3}", "fl", "id, " + fieldName, "sort", "id desc"), "//*[@numFound='3']", - "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0']"); + "0=count(//result/doc/" + type + "[@name='" + fieldName + "'][.='3'])", + "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='2']", + "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1']", + "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='0']"); - assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0']", "//result/doc[10]/" + type + "[@name='" + fieldName + "'][.='9']"); - assertQ(req("q", fieldName + ":[0 TO 1] OR " + fieldName + ":[8 TO 9]" , "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[0 TO 1] OR " + fieldName + ":[8 TO 9]" , "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1']", @@ -1114,7 +1121,7 @@ public class TestPointFields extends SolrTestCaseJ4 { } private void doTestIntPointFunctionQuery(String dvFieldName, String nonDvFieldName, String type) throws Exception { - for (int i = 0; i < 10; i++) { + for (int i = 9; i >= 0; i--) { assertU(adoc("id", String.valueOf(i), dvFieldName, String.valueOf(i), nonDvFieldName, String.valueOf(i))); } assertU(commit()); @@ -1127,14 +1134,14 @@ public class TestPointFields extends SolrTestCaseJ4 { "//result/doc[3]/" + type + "[@name='" + dvFieldName + "'][.='7']", "//result/doc[10]/" + type + "[@name='" + dvFieldName + "'][.='0']"); - assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", product(-1," + dvFieldName + ")"), + assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", product(-1," + dvFieldName + ")", "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/float[@name='product(-1," + dvFieldName + ")'][.='-0.0']", "//result/doc[2]/float[@name='product(-1," + dvFieldName + ")'][.='-1.0']", "//result/doc[3]/float[@name='product(-1," + dvFieldName + ")'][.='-2.0']", "//result/doc[10]/float[@name='product(-1," + dvFieldName + ")'][.='-9.0']"); - assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", field(" + dvFieldName + ")"), + assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", field(" + dvFieldName + ")", "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/" + type + "[@name='field(" + dvFieldName + ")'][.='0']", "//result/doc[2]/" + type + "[@name='field(" + dvFieldName + ")'][.='1']", @@ -1210,7 +1217,7 @@ public class TestPointFields extends SolrTestCaseJ4 { assert numbers != null && numbers.length == 20; assertTrue(h.getCore().getLatestSchema().getField(fieldName).multiValued()); assertTrue(h.getCore().getLatestSchema().getField(fieldName).getType() instanceof PointField); - for (int i=0; i < 10; i++) { + for (int i=9; i >= 0; i--) { assertU(adoc("id", String.valueOf(i), fieldName, numbers[i], fieldName, numbers[i+10])); } // Check using RTG before commit @@ -1232,27 +1239,26 @@ public class TestPointFields extends SolrTestCaseJ4 { "count(//doc/arr[@name='" + fieldName + "']/" + type + ")=2"); } } - String[] expected = new String[11]; - String[] expected2 = new String[11]; + String[] expected = new String[21]; expected[0] = "//*[@numFound='10']"; - expected2[0] = "//*[@numFound='10']"; for (int i = 1; i <= 10; i++) { + // checks for each doc's two values aren't next to eachother in array, but that doesn't matter for correctness expected[i] = "//result/doc[" + i + "]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[i-1] + "']"; - expected2[i] = "//result/doc[" + i + "]/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[i + 9] + "']"; + expected[i+10] = "//result/doc[" + i + "]/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[i + 9] + "']"; } - assertQ(req("q", "*:*", "fl", "id, " + fieldName), expected); - assertQ(req("q", "*:*", "fl", "id, " + fieldName), expected2); + assertQ(req("q", "*:*", "fl", "id, " + fieldName, "sort","id asc"), expected); } private void testPointFieldMultiValuedRangeQuery(String fieldName, String type, String[] numbers) throws Exception { assert numbers != null && numbers.length == 20; assertTrue(h.getCore().getLatestSchema().getField(fieldName).multiValued()); assertTrue(h.getCore().getLatestSchema().getField(fieldName).getType() instanceof PointField); - for (int i=0; i < 10; i++) { + for (int i=9; i >= 0; i--) { assertU(adoc("id", String.valueOf(i), fieldName, numbers[i], fieldName, numbers[i+10])); } assertU(commit()); - assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s]", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s]", fieldName, numbers[0], numbers[3]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[10] + "']", @@ -1263,52 +1269,61 @@ public class TestPointFields extends SolrTestCaseJ4 { "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[3] + "']", "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[13] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO %s]", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO %s]", fieldName, numbers[0], numbers[3]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']", "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']", "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[3] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s}", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s}", fieldName, numbers[0], numbers[3]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']", "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']", "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO %s}", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO %s}", fieldName, numbers[0], numbers[3]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='2']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']", "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO *}", fieldName, numbers[0]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO *}", fieldName, numbers[0]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO *}", fieldName, numbers[10]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO *}", fieldName, numbers[10]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='9']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:{* TO %s}", fieldName, numbers[3]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:{* TO %s}", fieldName, numbers[3]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:[* TO %s}", fieldName, numbers[3]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:[* TO %s}", fieldName, numbers[3]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']"); - assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']", "//result/doc[10]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[9] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s] OR %s:[%s TO %s]", fieldName, numbers[0], numbers[1], fieldName, numbers[8], numbers[9]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s] OR %s:[%s TO %s]", fieldName, numbers[0], numbers[1], fieldName, numbers[8], numbers[9]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']", "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']", "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[8] + "']", "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[9] + "']"); - assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s] OR %s:[%s TO %s]", fieldName, numbers[0], numbers[0], fieldName, numbers[10], numbers[10]), "fl", "id, " + fieldName), + assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s] OR %s:[%s TO %s]", fieldName, numbers[0], numbers[0], fieldName, numbers[10], numbers[10]), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='1']", "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']"); } @@ -1621,7 +1636,7 @@ public class TestPointFields extends SolrTestCaseJ4 { private void doTestPointFieldSort(String field, String dvField, String[] arr) throws Exception { assert arr != null && arr.length == 10; - for (int i = 0; i < arr.length; i++) { + for (int i = arr.length-1; i >= 0; i--) { assertU(adoc("id", String.valueOf(i), dvField, String.valueOf(arr[i]), field, String.valueOf(arr[i]))); } assertU(commit()); @@ -1645,47 +1660,51 @@ public class TestPointFields extends SolrTestCaseJ4 { } private void doTestFloatPointFieldRangeQuery(String fieldName, String type, boolean testDouble) throws Exception { - for (int i = 0; i < 10; i++) { + for (int i = 9; i >= 0; i--) { assertU(adoc("id", String.valueOf(i), fieldName, String.valueOf(i))); } assertU(commit()); - assertQ(req("q", fieldName + ":[0 TO 3]", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[0 TO 3]", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0.0']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1.0']", "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='2.0']", "//result/doc[4]/" + type + "[@name='" + fieldName + "'][.='3.0']"); - assertQ(req("q", fieldName + ":{0 TO 3]", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":{0 TO 3]", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='1.0']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='2.0']", "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='3.0']"); - assertQ(req("q", fieldName + ":[0 TO 3}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[0 TO 3}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0.0']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1.0']", "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='2.0']"); - assertQ(req("q", fieldName + ":{0 TO 3}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":{0 TO 3}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='2']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='1.0']", "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='2.0']"); - assertQ(req("q", fieldName + ":{0 TO *}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":{0 TO *}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='9']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='1.0']"); - assertQ(req("q", fieldName + ":{* TO 3}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":{* TO 3}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", - "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0.0']"); + "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0.0']", + "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1.0']", + "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='2.0']"); - assertQ(req("q", fieldName + ":[* TO 3}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[* TO 3}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", - "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0.0']"); + "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0.0']", + "//result/doc[2]/" + type + "[@name='" + fieldName + "'][.='1.0']", + "//result/doc[3]/" + type + "[@name='" + fieldName + "'][.='2.0']"); - assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/" + type + "[@name='" + fieldName + "'][.='0.0']", "//result/doc[10]/" + type + "[@name='" + fieldName + "'][.='9.0']"); @@ -1769,7 +1788,7 @@ public class TestPointFields extends SolrTestCaseJ4 { } private void doTestFloatPointFunctionQuery(String dvFieldName, String nonDvFieldName, String type) throws Exception { - for (int i = 0; i < 10; i++) { + for (int i = 9; i >= 0; i--) { assertU(adoc("id", String.valueOf(i), dvFieldName, String.format(Locale.ROOT, "%f", (float)i*1.1), nonDvFieldName, String.format(Locale.ROOT, "%f", (float)i*1.1))); } assertU(commit()); @@ -1782,14 +1801,14 @@ public class TestPointFields extends SolrTestCaseJ4 { "//result/doc[3]/" + type + "[@name='" + dvFieldName + "'][.='7.7']", "//result/doc[10]/" + type + "[@name='" + dvFieldName + "'][.='0.0']"); - assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", product(-1," + dvFieldName + ")"), + assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", product(-1," + dvFieldName + ")", "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/float[@name='product(-1," + dvFieldName + ")'][.='-0.0']", "//result/doc[2]/float[@name='product(-1," + dvFieldName + ")'][.='-1.1']", "//result/doc[3]/float[@name='product(-1," + dvFieldName + ")'][.='-2.2']", "//result/doc[10]/float[@name='product(-1," + dvFieldName + ")'][.='-9.9']"); - assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", field(" + dvFieldName + ")"), + assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", field(" + dvFieldName + ")", "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/" + type + "[@name='field(" + dvFieldName + ")'][.='0.0']", "//result/doc[2]/" + type + "[@name='field(" + dvFieldName + ")'][.='1.1']", @@ -2014,53 +2033,61 @@ public class TestPointFields extends SolrTestCaseJ4 { private void doTestDatePointFieldRangeQuery(String fieldName) throws Exception { String baseDate = "1995-12-31T10:59:59Z"; - for (int i = 0; i < 10; i++) { + for (int i = 9; i >= 0; i--) { assertU(adoc("id", String.valueOf(i), fieldName, String.format(Locale.ROOT, "%s+%dHOURS", baseDate, i))); } assertU(commit()); - assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[%s+0HOURS TO %s+3HOURS]", baseDate, baseDate), "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[%s+0HOURS TO %s+3HOURS]", baseDate, baseDate), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']", "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']", "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']", "//result/doc[4]/date[@name='" + fieldName + "'][.='1995-12-31T13:59:59Z']"); - assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{%s+0HOURS TO %s+3HOURS]", baseDate, baseDate), "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{%s+0HOURS TO %s+3HOURS]", baseDate, baseDate), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']", "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']", "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T13:59:59Z']"); - assertQ(req("q", fieldName + ":"+ String.format(Locale.ROOT, "[%s+0HOURS TO %s+3HOURS}",baseDate,baseDate), "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":"+ String.format(Locale.ROOT, "[%s+0HOURS TO %s+3HOURS}",baseDate,baseDate), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']", "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']", "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']"); - assertQ(req("q", fieldName + ":"+ String.format(Locale.ROOT, "{%s+0HOURS TO %s+3HOURS}",baseDate,baseDate), "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":"+ String.format(Locale.ROOT, "{%s+0HOURS TO %s+3HOURS}",baseDate,baseDate), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='2']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']", "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']"); - assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{%s+0HOURS TO *}",baseDate), "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{%s+0HOURS TO *}",baseDate), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='9']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']"); - assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{* TO %s+3HOURS}",baseDate), "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{* TO %s+3HOURS}",baseDate), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']"); - assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[* TO %s+3HOURS}",baseDate), "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[* TO %s+3HOURS}",baseDate), + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='3']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']"); - assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName), + assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']", "//result/doc[10]/date[@name='" + fieldName + "'][.='1995-12-31T19:59:59Z']"); assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[%s+0HOURS TO %s+1HOURS]",baseDate,baseDate) - +" OR " + fieldName + ":" + String.format(Locale.ROOT, "[%s+8HOURS TO %s+9HOURS]",baseDate,baseDate) , "fl", "id, " + fieldName), + + " OR " + fieldName + ":" + String.format(Locale.ROOT, "[%s+8HOURS TO %s+9HOURS]",baseDate,baseDate) , + "fl", "id, " + fieldName, "sort", "id asc"), "//*[@numFound='4']", "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']", "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']", @@ -2218,28 +2245,28 @@ public class TestPointFields extends SolrTestCaseJ4 { private void doTestDatePointFunctionQuery(String dvFieldName, String nonDvFieldName, String type) throws Exception { String baseDate = "1995-01-10T10:59:10Z"; - for (int i = 0; i < 10; i++) { + for (int i = 9; i >= 0; i--) { String date = String.format(Locale.ROOT, "%s+%dSECONDS", baseDate, i+1); assertU(adoc("id", String.valueOf(i), dvFieldName, date, nonDvFieldName, date)); } assertU(commit()); assertTrue(h.getCore().getLatestSchema().getField(dvFieldName).hasDocValues()); assertTrue(h.getCore().getLatestSchema().getField(dvFieldName).getType() instanceof PointField); - assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "sort", "product(-1," + dvFieldName + ") asc"), + assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "sort", "product(-1,ms(" + dvFieldName + ")) asc"), "//*[@numFound='10']", - "//result/doc[1]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:11Z']", - "//result/doc[2]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:12Z']", - "//result/doc[3]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:13Z']", - "//result/doc[10]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:20Z']"); + "//result/doc[1]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:20Z']", + "//result/doc[2]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:19Z']", + "//result/doc[3]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:18Z']", + "//result/doc[10]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:11Z']"); - assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", ms(" + dvFieldName + ","+baseDate+")"), + assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", ms(" + dvFieldName + ","+baseDate+")", "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='1000.0']", "//result/doc[2]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='2000.0']", "//result/doc[3]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='3000.0']", "//result/doc[10]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='10000.0']"); - assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", field(" + dvFieldName + ")"), + assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", field(" + dvFieldName + ")", "sort", "id asc"), "//*[@numFound='10']", "//result/doc[1]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:11Z']", "//result/doc[2]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:12Z']", From 40c5fd87f3d6ca09c92c42447b338dda525d9eb7 Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Tue, 14 Mar 2017 10:23:49 -0700 Subject: [PATCH 09/20] SOLR-10184: Fix bin/solr so it can run properly on java9 --- solr/CHANGES.txt | 2 ++ solr/bin/solr | 57 ++++++++++++++++++++++++++++----------------- solr/bin/solr.in.sh | 11 +++++++-- 3 files changed, 47 insertions(+), 23 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 94e52572066..5f9417130cc 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -249,6 +249,8 @@ Bug Fixes legacyCloud=false and will also fail on a state check when taking over a core registration with a new core. (Mark Miller, Hrishikesh Gadre, Patrick Dvorack) +* SOLR-10184: Fix bin/solr so it can run properly on java9 (hossman, Uwe Schindler) + Optimizations ---------------------- diff --git a/solr/bin/solr b/solr/bin/solr index cd9db0f3355..9563bb2df93 100755 --- a/solr/bin/solr +++ b/solr/bin/solr @@ -50,7 +50,7 @@ verbose=false THIS_OS=`uname -s` # What version of Java is required to run this version of Solr. -JAVA_VER_REQ="8" # For printing in echo +JAVA_VER_REQ="1.8" stop_all=false @@ -130,27 +130,34 @@ if [[ $? -ne 0 ]] ; then echo >&2 "Please install latest version of Java $JAVA_VER_REQ or set JAVA_HOME properly." echo >&2 "Command that we tried: '${JAVA} -version', with response:" echo >&2 "${JAVA_VER}" - echo + echo >&2 echo >&2 "Debug information:" echo >&2 "JAVA_HOME: ${JAVA_HOME:-N/A}" echo >&2 "Active Path:" echo >&2 "${PATH}" exit 1 else - JAVA_VER=$(echo $JAVA_VER | awk -F '"' '/version/ {print $2}') - if [[ "$JAVA_VER" < "1.$JAVA_VER_REQ" ]] ; then + JAVA_VER_NUM=$(echo $JAVA_VER | head -1 | awk -F '"' '/version/ {print $2}') + if [[ "$JAVA_VER_NUM" < "$JAVA_VER_REQ" ]] ; then echo >&2 "Your current version of Java is too old to run this version of Solr" - echo >&2 "We found version $JAVA_VER, using command '${JAVA}'" + echo >&2 "We found version $JAVA_VER_NUM, using command '${JAVA} -version', with response:" + echo >&2 "${JAVA_VER}" + echo >&2 echo >&2 "Please install latest version of Java $JAVA_VER_REQ or set JAVA_HOME properly." - echo + echo >&2 echo >&2 "Debug information:" echo >&2 "JAVA_HOME: ${JAVA_HOME:-N/A}" echo >&2 "Active Path:" echo >&2 "${PATH}" exit 1 fi + JAVA_VENDOR="Oracle" + if [ "`echo $JAVA_VER | grep -i "IBM J9"`" != "" ]; then + JAVA_VENDOR="IBM J9" + fi fi + # Select HTTP OR HTTPS related configurations SOLR_URL_SCHEME=http SOLR_JETTY_CONFIG=() @@ -1519,29 +1526,37 @@ if [ "${SOLR_LOG_PRESTART_ROTATION:=true}" == "true" ]; then run_tool utils -s "$DEFAULT_SERVER_DIR" -l "$SOLR_LOGS_DIR" $q -rotate_solr_logs 9 || echo "Failed rotating old solr logs" fi -java_ver_out=`echo "$("$JAVA" -version 2>&1)"` -JAVA_VERSION=`echo $java_ver_out | grep "java version" | awk '{ print substr($3, 2, length($3)-2); }'` -JAVA_VENDOR="Oracle" -if [ "`echo $java_ver_out | grep -i "IBM J9"`" != "" ]; then - JAVA_VENDOR="IBM J9" -fi - -# Establish default opts no env var set (otherwise init to empty) +# Establish default GC logging opts if no env var set (otherwise init to sensible default) if [ -z ${GC_LOG_OPTS+x} ]; then - GC_LOG_OPTS=('-verbose:gc' '-XX:+PrintHeapAtGC' '-XX:+PrintGCDetails' \ - '-XX:+PrintGCDateStamps' '-XX:+PrintGCTimeStamps' '-XX:+PrintTenuringDistribution' \ - '-XX:+PrintGCApplicationStoppedTime') + if [[ "$JAVA_VER_NUM" < "9" ]] ; then + GC_LOG_OPTS=('-verbose:gc' '-XX:+PrintHeapAtGC' '-XX:+PrintGCDetails' \ + '-XX:+PrintGCDateStamps' '-XX:+PrintGCTimeStamps' '-XX:+PrintTenuringDistribution' \ + '-XX:+PrintGCApplicationStoppedTime') + else + GC_LOG_OPTS=('-Xlog:gc*') + fi else GC_LOG_OPTS=($GC_LOG_OPTS) fi # if verbose gc logging enabled, setup the location of the log file and rotation if [ "$GC_LOG_OPTS" != "" ]; then - gc_log_flag="-Xloggc" - if [ "$JAVA_VENDOR" == "IBM J9" ]; then - gc_log_flag="-Xverbosegclog" + if [[ "$JAVA_VER_NUM" < "9" ]] ; then + gc_log_flag="-Xloggc" + if [ "$JAVA_VENDOR" == "IBM J9" ]; then + gc_log_flag="-Xverbosegclog" + fi + GC_LOG_OPTS+=("$gc_log_flag:$SOLR_LOGS_DIR/solr_gc.log" '-XX:+UseGCLogFileRotation' '-XX:NumberOfGCLogFiles=9' '-XX:GCLogFileSize=20M') + else + # http://openjdk.java.net/jeps/158 + for i in "${!GC_LOG_OPTS[@]}"; + do + # for simplicity, we only look at the prefix '-Xlog:gc' + # (if 'all' or multiple tags are used starting with anything other then 'gc' the user is on their own) + # if a single additional ':' exists in param, then there is already an explicit output specifier + GC_LOG_OPTS[$i]=$(echo ${GC_LOG_OPTS[$i]} | sed "s|^\(-Xlog:gc[^:]*$\)|\1:file=$SOLR_LOGS_DIR/solr_gc.log:time,uptime:filecount=9,filesize=20000|") + done fi - GC_LOG_OPTS+=("$gc_log_flag:$SOLR_LOGS_DIR/solr_gc.log" -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=9 -XX:GCLogFileSize=20M) fi # If ZK_HOST is defined, the assume SolrCloud mode diff --git a/solr/bin/solr.in.sh b/solr/bin/solr.in.sh index 878702f4c5a..67839f95cf1 100644 --- a/solr/bin/solr.in.sh +++ b/solr/bin/solr.in.sh @@ -34,9 +34,16 @@ # Comment out SOLR_HEAP if you are using this though, that takes precedence #SOLR_JAVA_MEM="-Xms512m -Xmx512m" -# Enable verbose GC logging +# Enable verbose GC logging... +# * If this is unset, various default options will be selected depending on which JVM version is in use +# * For java8 or lower: if this is set, additional params will be added to specify the log file & rotation +# * For java9 or higher: each included opt param that starts with '-Xlog:gc', but does not include an output +# specifier, will have a 'file' output specifier (as well as formatting & rollover options) appended, +# using the effective value of the SOLR_LOGS_DIR. +# +#GC_LOG_OPTS='-Xlog:gc*' # (java9) #GC_LOG_OPTS="-verbose:gc -XX:+PrintHeapAtGC -XX:+PrintGCDetails \ -#-XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCApplicationStoppedTime" +# -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCApplicationStoppedTime" # These GC settings have shown to work well for a number of common Solr workloads #GC_TUNE="-XX:NewRatio=3 -XX:SurvivorRatio=4 etc. From 533f968f812054252a8c20ff892d49418c4f219f Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 14 Mar 2017 19:57:57 +0100 Subject: [PATCH 10/20] SOLR-10224: Add disk total and disk free metrics. --- solr/CHANGES.txt | 2 ++ .../src/java/org/apache/solr/core/CoreContainer.java | 7 ++++++- solr/core/src/java/org/apache/solr/core/SolrCore.java | 7 +++++++ .../metrics/reporters/solr/SolrClusterReporter.java | 10 ++++++---- .../solr/metrics/reporters/solr/SolrShardReporter.java | 5 +++-- 5 files changed, 24 insertions(+), 7 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 5f9417130cc..b3ce30cbf08 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -183,6 +183,8 @@ New Features * SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs. (Mano Kovacs via Mark Miller) +* SOLR-10224: Add disk total and disk free metrics. (ab) + Bug Fixes ---------------------- diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java index 2da48b828e3..9e22f91b49c 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -530,7 +530,8 @@ public class CoreContainer { containerProperties.putAll(cfg.getSolrProperties()); - // initialize gauges for reporting the number of cores + // initialize gauges for reporting the number of cores and disk total/free + String registryName = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node); metricManager.registerGauge(registryName, () -> solrCores.getCores().size(), true, "loaded", SolrInfoMBean.Category.CONTAINER.toString(), "cores"); @@ -538,6 +539,10 @@ public class CoreContainer { true, "lazy",SolrInfoMBean.Category.CONTAINER.toString(), "cores"); metricManager.registerGauge(registryName, () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size(), true, "unloaded",SolrInfoMBean.Category.CONTAINER.toString(), "cores"); + metricManager.registerGauge(registryName, () -> cfg.getCoreRootDirectory().toFile().getTotalSpace(), + true, "totalSpace", SolrInfoMBean.Category.CONTAINER.toString(), "fs"); + metricManager.registerGauge(registryName, () -> cfg.getCoreRootDirectory().toFile().getUsableSpace(), + true, "usableSpace", SolrInfoMBean.Category.CONTAINER.toString(), "fs"); if (isZooKeeperAware()) { metricManager.loadClusterReporters(cfg.getMetricReporterPlugins(), this); diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java index bc41b1900f0..6e25280c653 100644 --- a/solr/core/src/java/org/apache/solr/core/SolrCore.java +++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java @@ -30,6 +30,8 @@ import java.lang.reflect.Constructor; import java.net.URL; import java.nio.charset.StandardCharsets; import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -1137,6 +1139,11 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea manager.registerGauge(registry, () -> getIndexSize(), true, "sizeInBytes", Category.INDEX.toString()); manager.registerGauge(registry, () -> NumberUtils.readableSize(getIndexSize()), true, "size", Category.INDEX.toString()); manager.registerGauge(registry, () -> coreDescriptor.getCoreContainer().getCoreNames(this), true, "aliases", Category.CORE.toString()); + // initialize disk total / free metrics + Path dataDirPath = Paths.get(dataDir); + File dataDirFile = dataDirPath.toFile(); + manager.registerGauge(registry, () -> dataDirFile.getTotalSpace(), true, "totalSpace", Category.CORE.toString(), "fs"); + manager.registerGauge(registry, () -> dataDirFile.getUsableSpace(), true, "usableSpace", Category.CORE.toString(), "fs"); } private Map initInfoRegistry(String name, SolrConfig config) { diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java index 846e805cf68..0c3b651a573 100644 --- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java +++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java @@ -106,10 +106,12 @@ public class SolrClusterReporter extends SolrMetricReporter { add("os\\.FreeSwapSpaceSize"); add("os\\.OpenFileDescriptorCount"); add("threads\\.count"); - }})); // all metrics - // XXX anything interesting here? - //add(new SolrReporter.Specification(OVERSEER_GROUP, "node", SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.node.toString()), - // Collections.emptySet())); // all metrics + }})); + add(new SolrReporter.Report(CLUSTER_GROUP, "node", SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.node.toString()), + new HashSet() {{ + add("CONTAINER\\.cores\\..*"); + add("CONTAINER\\.fs\\..*"); + }})); add(new SolrReporter.Report(CLUSTER_GROUP, "leader.$1", "solr\\.collection\\.(.*)\\.leader", new HashSet(){{ add("UPDATE\\./update/.*"); diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java index 2b202749504..8b36d3e0c96 100644 --- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java +++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java @@ -61,9 +61,10 @@ public class SolrShardReporter extends SolrMetricReporter { public static final List DEFAULT_FILTERS = new ArrayList(){{ add("TLOG.*"); + add("CORE\\.fs.*"); add("REPLICATION.*"); - add("INDEX.flush.*"); - add("INDEX.merge.major.*"); + add("INDEX\\.flush.*"); + add("INDEX\\.merge\\.major.*"); add("UPDATE\\./update/.*requests"); add("QUERY\\./select.*requests"); }}; From e66924cc5c58c24148236aa22858bf1aa9fd9240 Mon Sep 17 00:00:00 2001 From: Noble Paul Date: Wed, 15 Mar 2017 18:38:48 +1030 Subject: [PATCH 11/20] SOLR-8045: SerachHandler to override the registerV2() and remove registerPath from sample solrconfig.xml --- .../org/apache/solr/handler/component/SearchHandler.java | 5 +++++ .../sample_techproducts_configs/conf/solrconfig.xml | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java index ba581d43131..c05c6c42670 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java @@ -478,6 +478,11 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware , } return sb.toString(); } + + @Override + public Boolean registerV2() { + return Boolean.TRUE; + } } diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml index 3ff89c0307d..23c2acd4934 100644 --- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml +++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml @@ -850,7 +850,7 @@ - + explicit json From 9f72056d94c47344dbba30b282b30f0e4b70256f Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 15 Mar 2017 06:03:54 -0400 Subject: [PATCH 12/20] LUCENE-7742: fix places where we were unboxing and then re-boxing according to FindBugs --- lucene/CHANGES.txt | 3 +++ .../org/apache/lucene/document/LatLonDocValuesField.java | 2 +- .../org/apache/lucene/spatial3d/Geo3DDocValuesField.java | 2 +- .../processor/TikaLanguageIdentifierUpdateProcessor.java | 5 +++-- solr/core/src/java/org/apache/solr/core/RequestParams.java | 2 +- .../src/java/org/apache/solr/handler/sql/SolrEnumerator.java | 4 ++-- .../src/java/org/apache/solr/schema/NumericFieldType.java | 4 ++-- solr/core/src/java/org/apache/solr/search/Grouping.java | 2 +- .../src/java/org/apache/solr/search/SolrIndexSearcher.java | 2 +- .../responseprocessor/SearchGroupShardResponseProcessor.java | 2 +- .../org/apache/solr/security/AutorizationEditOperation.java | 2 +- .../org/apache/solr/search/TestCollapseQParserPlugin.java | 2 +- .../org/apache/solr/update/TestInPlaceUpdatesDistrib.java | 2 +- .../apache/solr/client/solrj/io/eval/EqualsEvaluator.java | 2 +- .../src/java/org/apache/solr/common/cloud/DocCollection.java | 2 +- .../test/org/apache/solr/common/params/SolrParamTest.java | 4 ++-- 16 files changed, 23 insertions(+), 19 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index e14ab53e41d..62f4763325e 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -256,6 +256,9 @@ Optimizations * LUCENE-7699: Query parsers now use span queries to produce more efficient phrase queries for multi-token synonyms. (Matt Webber via Jim Ferenczi) +* LUCENE-7742: Fix places where we were unboxing and then re-boxing + according to FindBugs (Daniel Jelinski via Mike McCandless) + Build * LUCENE-7653: Update randomizedtesting to version 2.5.0. (Dawid Weiss) diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonDocValuesField.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonDocValuesField.java index 08a7da78012..10e90b4838b 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonDocValuesField.java +++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonDocValuesField.java @@ -106,7 +106,7 @@ public class LatLonDocValuesField extends Field { result.append(name); result.append(':'); - long currentValue = Long.valueOf((Long)fieldsData); + long currentValue = (Long)fieldsData; result.append(decodeLatitude((int)(currentValue >> 32))); result.append(','); result.append(decodeLongitude((int)(currentValue & 0xFFFFFFFF))); diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DDocValuesField.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DDocValuesField.java index 551fa5a8e4f..dc9df35576b 100644 --- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DDocValuesField.java +++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DDocValuesField.java @@ -305,7 +305,7 @@ public class Geo3DDocValuesField extends Field { result.append(name); result.append(':'); - long currentValue = Long.valueOf((Long)fieldsData); + long currentValue = (Long)fieldsData; result.append(decodeXValue(currentValue)); result.append(','); diff --git a/solr/contrib/langid/src/java/org/apache/solr/update/processor/TikaLanguageIdentifierUpdateProcessor.java b/solr/contrib/langid/src/java/org/apache/solr/update/processor/TikaLanguageIdentifierUpdateProcessor.java index 3d3fa41be00..836a3bf6795 100644 --- a/solr/contrib/langid/src/java/org/apache/solr/update/processor/TikaLanguageIdentifierUpdateProcessor.java +++ b/solr/contrib/langid/src/java/org/apache/solr/update/processor/TikaLanguageIdentifierUpdateProcessor.java @@ -56,8 +56,9 @@ public class TikaLanguageIdentifierUpdateProcessor extends LanguageIdentifierUpd // FIXME: Hack - we get the distance from toString and calculate our own certainty score Double distance = Double.parseDouble(tikaSimilarityPattern.matcher(identifier.toString()).replaceFirst("$1")); // This formula gives: 0.02 => 0.8, 0.1 => 0.5 which is a better sweetspot than isReasonablyCertain() - Double certainty = 1 - (5 * distance); - certainty = (certainty < 0) ? 0 : certainty; + Double certainty = 1 - (5 * distance); + if (certainty < 0) + certainty = 0d; DetectedLanguage language = new DetectedLanguage(identifier.getLanguage(), certainty); languages.add(language); log.debug("Language detected as "+language+" with a certainty of "+language.getCertainty()+" (Tika distance="+identifier.toString()+")"); diff --git a/solr/core/src/java/org/apache/solr/core/RequestParams.java b/solr/core/src/java/org/apache/solr/core/RequestParams.java index ff0d36c5685..fbb2555e322 100644 --- a/solr/core/src/java/org/apache/solr/core/RequestParams.java +++ b/solr/core/src/java/org/apache/solr/core/RequestParams.java @@ -222,7 +222,7 @@ public class RequestParams implements MapSerializable { } public Long getVersion() { - return meta == null ? 0l : (Long) meta.get("v"); + return meta == null ? Long.valueOf(0l) : (Long) meta.get("v"); } @Override diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrEnumerator.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrEnumerator.java index 6f9dddfbf2f..be6046c98fe 100644 --- a/solr/core/src/java/org/apache/solr/handler/sql/SolrEnumerator.java +++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrEnumerator.java @@ -103,10 +103,10 @@ class SolrEnumerator implements Enumerator { private Object getRealVal(Object val) { // Check if Double is really a Long if(val instanceof Double) { - Double doubleVal = (double) val; + double doubleVal = (double) val; //make sure that double has no decimals and fits within Long if(doubleVal % 1 == 0 && doubleVal >= Long.MIN_VALUE && doubleVal <= Long.MAX_VALUE) { - return doubleVal.longValue(); + return (long)doubleVal; } return doubleVal; } diff --git a/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java b/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java index 5801766f7a9..6cda9ca6438 100644 --- a/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java +++ b/solr/core/src/java/org/apache/solr/schema/NumericFieldType.java @@ -114,10 +114,10 @@ public abstract class NumericFieldType extends PrimitiveFieldType { if ((minVal == null || minVal.doubleValue() < 0d || minBits == minusZeroBits) && (maxVal != null && (maxVal.doubleValue() < 0d || maxBits == minusZeroBits))) { query = numericDocValuesRangeQuery - (fieldName, maxBits, (min == null ? negativeInfinityBits : minBits), maxInclusive, minInclusive, false); + (fieldName, maxBits, (min == null ? Long.valueOf(negativeInfinityBits) : minBits), maxInclusive, minInclusive, false); } else { // If both max and min are positive, then issue range query query = numericDocValuesRangeQuery - (fieldName, minBits, (max == null ? positiveInfinityBits : maxBits), minInclusive, maxInclusive, false); + (fieldName, minBits, (max == null ? Long.valueOf(positiveInfinityBits) : maxBits), minInclusive, maxInclusive, false); } } return query; diff --git a/solr/core/src/java/org/apache/solr/search/Grouping.java b/solr/core/src/java/org/apache/solr/search/Grouping.java index 327e5bb6e44..302383a00a7 100644 --- a/solr/core/src/java/org/apache/solr/search/Grouping.java +++ b/solr/core/src/java/org/apache/solr/search/Grouping.java @@ -601,7 +601,7 @@ public class Grouping { groupResult.add("matches", matches); if (totalCount == TotalCount.grouped) { Integer totalNrOfGroups = getNumberOfGroups(); - groupResult.add("ngroups", totalNrOfGroups == null ? 0 : totalNrOfGroups); + groupResult.add("ngroups", totalNrOfGroups == null ? Integer.valueOf(0) : totalNrOfGroups); } maxMatches = Math.max(maxMatches, matches); return groupResult; diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java index a7ee433f4d1..19e47d0a022 100644 --- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java +++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java @@ -847,7 +847,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI newVal = val.intValue(); break; case LONG: - newVal = val.longValue(); + newVal = val; break; case FLOAT: newVal = Float.intBitsToFloat(val.intValue()); diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java index 1645b1edd49..a12cad1912c 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java @@ -119,7 +119,7 @@ public class SearchGroupShardResponseProcessor implements ShardResponseProcessor if (groupCount != null) { Integer existingGroupCount = rb.mergedGroupCounts.get(field); // Assuming groups don't cross shard boundary... - rb.mergedGroupCounts.put(field, existingGroupCount != null ? existingGroupCount + groupCount : groupCount); + rb.mergedGroupCounts.put(field, existingGroupCount != null ? Integer.valueOf(existingGroupCount + groupCount) : groupCount); } final Collection> searchGroups = firstPhaseCommandResult.getSearchGroups(); diff --git a/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java b/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java index 88c7987ef48..6a5230cd9c0 100644 --- a/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java +++ b/solr/core/src/java/org/apache/solr/security/AutorizationEditOperation.java @@ -83,7 +83,7 @@ enum AutorizationEditOperation { boolean indexSatisfied = index == null; for (int i = 0; i < permissions.size(); i++) { Map perm = permissions.get(i); - Integer thisIdx = (int) perm.get("index"); + Integer thisIdx = (Integer) perm.get("index"); if (thisIdx.equals(beforeIdx)) { beforeSatisfied = true; permissionsCopy.add(dataMap); diff --git a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java index 2fca452a520..ea4114a08b0 100644 --- a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java +++ b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java @@ -336,7 +336,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { if(boostedResults.size() == controlResults.size()) { for(int i=0; i { this.replicationFactor = (Integer) verifyProp(props, REPLICATION_FACTOR); this.maxShardsPerNode = (Integer) verifyProp(props, MAX_SHARDS_PER_NODE); Boolean autoAddReplicas = (Boolean) verifyProp(props, AUTO_ADD_REPLICAS); - this.autoAddReplicas = autoAddReplicas == null ? false : autoAddReplicas; + this.autoAddReplicas = autoAddReplicas == null ? Boolean.FALSE : autoAddReplicas; Integer realtimeReplicas = (Integer) verifyProp(props, REALTIME_REPLICAS); this.realtimeReplicas = realtimeReplicas == null ? -1 : realtimeReplicas; if (this.realtimeReplicas != -1 && this.realtimeReplicas != 1) { diff --git a/solr/solrj/src/test/org/apache/solr/common/params/SolrParamTest.java b/solr/solrj/src/test/org/apache/solr/common/params/SolrParamTest.java index 48237c5250b..80f9036433b 100644 --- a/solr/solrj/src/test/org/apache/solr/common/params/SolrParamTest.java +++ b/solr/solrj/src/test/org/apache/solr/common/params/SolrParamTest.java @@ -213,10 +213,10 @@ public class SolrParamTest extends LuceneTestCase { // Get things with defaults assertEquals( pstr , params.get( "xxx", pstr ) ); - assertEquals( pbool.booleanValue() , params.getBool( "xxx", pbool ) ); + assertEquals( pbool , params.getBool( "xxx", pbool ) ); assertEquals( pint.intValue() , params.getInt( "xxx", pint ) ); assertEquals( pfloat.floatValue() , params.getFloat( "xxx", pfloat ), 0.1); - assertEquals( pbool.booleanValue() , params.getFieldBool( "xxx", "bool", pbool ) ); + assertEquals( pbool , params.getFieldBool( "xxx", "bool", pbool ) ); assertEquals( pint.intValue() , params.getFieldInt( "xxx", "int", pint ) ); assertEquals( pfloat.floatValue() , params.getFieldFloat("xxx", "float", pfloat ), 0.1); assertEquals( pstr , params.getFieldParam("xxx", "str", pstr ) ); From 10b7be59f71de5789e47eee60426018dc342e333 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 15 Mar 2017 06:08:10 -0400 Subject: [PATCH 13/20] LUCENE-7739: Fix places where we unnecessarily boxed while parsing a numeric value according to FindBugs --- lucene/CHANGES.txt | 4 ++++ .../byTask/feeds/EnwikiContentSource.java | 2 +- .../benchmark/byTask/tasks/ForceMergeTask.java | 2 +- .../org/apache/lucene/util/fst/TestFSTs.java | 4 ++-- .../apache/lucene/search/join/TestBlockJoin.java | 2 +- .../search/TestDiversifiedTopDocsCollector.java | 2 +- .../queries/function/TestValueSources.java | 6 +++--- .../queryparser/classic/QueryParserBase.java | 6 +++--- .../standard/parser/StandardSyntaxParser.java | 8 ++++---- .../standard/parser/StandardSyntaxParser.jj | 8 ++++---- .../queryparser/surround/parser/QueryParser.java | 2 +- .../queryparser/surround/parser/QueryParser.jj | 2 +- .../xml/builders/PointRangeQueryBuilder.java | 16 ++++++++-------- .../queryparser/classic/TestQueryParser.java | 2 +- .../queryparser/xml/CoreParserTestIndexData.java | 2 +- .../analytics/util/RangeEndpointCalculator.java | 8 ++++---- .../handler/dataimport/MailEntityProcessor.java | 2 +- .../java/org/apache/solr/response/PageTool.java | 2 +- .../org/apache/solr/handler/IndexFetcher.java | 6 +++--- .../apache/solr/handler/ReplicationHandler.java | 2 +- .../handler/component/RangeFacetRequest.java | 8 ++++---- .../apache/solr/parser/SolrQueryParserBase.java | 4 ++-- .../org/apache/solr/search/facet/FacetRange.java | 8 ++++---- .../TolerantUpdateProcessorFactory.java | 2 +- .../org/apache/solr/util/DateMathParser.java | 2 +- .../org/apache/solr/util/SolrPluginUtils.java | 4 ++-- .../core/snapshots/TestSolrCloudSnapshots.java | 2 +- .../core/snapshots/TestSolrCoreSnapshots.java | 2 +- .../solr/search/TestSolrFieldCacheMBean.java | 4 ++-- .../solr/search/mlt/CloudMLTQParserTest.java | 14 +++++++------- 30 files changed, 71 insertions(+), 67 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 62f4763325e..bd38f3f30cb 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -259,6 +259,10 @@ Optimizations * LUCENE-7742: Fix places where we were unboxing and then re-boxing according to FindBugs (Daniel Jelinski via Mike McCandless) +* LUCENE-7739: Fix places where we unnecessarily boxed while parsing + a numeric value according to FindBugs (Daniel Jelinski via Mike + McCandless) + Build * LUCENE-7653: Update randomizedtesting to version 2.5.0. (Dawid Weiss) diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java index a933e562789..7258476357a 100644 --- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java +++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java @@ -101,7 +101,7 @@ public class EnwikiContentSource extends ContentSource { buffer.append(original.substring(8, 10)); buffer.append('-'); - buffer.append(months[Integer.valueOf(original.substring(5, 7)).intValue() - 1]); + buffer.append(months[Integer.parseInt(original.substring(5, 7)) - 1]); buffer.append('-'); buffer.append(original.substring(0, 4)); buffer.append(' '); diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ForceMergeTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ForceMergeTask.java index 40a3637750e..292642f3607 100644 --- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ForceMergeTask.java +++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ForceMergeTask.java @@ -46,7 +46,7 @@ public class ForceMergeTask extends PerfTask { @Override public void setParams(String params) { super.setParams(params); - maxNumSegments = Double.valueOf(params).intValue(); + maxNumSegments = (int)Double.parseDouble(params); } @Override diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java index dcce285383b..7a8e5f5f6e9 100644 --- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java +++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java @@ -624,10 +624,10 @@ public class TestFSTs extends LuceneTestCase { int idx = 0; while (idx < args.length) { if (args[idx].equals("-prune")) { - prune = Integer.valueOf(args[1 + idx]); + prune = Integer.parseInt(args[1 + idx]); idx++; } else if (args[idx].equals("-limit")) { - limit = Integer.valueOf(args[1 + idx]); + limit = Integer.parseInt(args[1 + idx]); idx++; } else if (args[idx].equals("-utf8")) { inputMode = 0; diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java index a13e66fc21a..da3c20ed578 100644 --- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java +++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java @@ -1011,7 +1011,7 @@ public class TestBlockJoin extends LuceneTestCase { TopDocs childHits = new TopDocs(0, new ScoreDoc[0], 0f); for (ScoreDoc controlHit : controlHits.scoreDocs) { Document controlDoc = r.document(controlHit.doc); - int parentID = Integer.valueOf(controlDoc.get("parentID")); + int parentID = Integer.parseInt(controlDoc.get("parentID")); if (parentID != currentParentID) { assertEquals(childHitSlot, childHits.scoreDocs.length); currentParentID = parentID; diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java index 043141ac72a..f07793ace6d 100644 --- a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java +++ b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java @@ -392,7 +392,7 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase { for (int i = 0; i < hitsOfThe60s.length; i++) { String cols[] = hitsOfThe60s[i].split("\t"); Record record = new Record(String.valueOf(i), cols[0], cols[1], cols[2], - Float.valueOf(cols[3])); + Float.parseFloat(cols[3])); parsedRecords.put(record.id, record); idField.setStringValue(record.id); yearField.setStringValue(record.year); diff --git a/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java b/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java index e0082932559..800859036eb 100644 --- a/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java +++ b/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java @@ -123,9 +123,9 @@ public class TestValueSources extends LuceneTestCase { document.add(new StringField("id", doc[0], Field.Store.NO)); document.add(new SortedDocValuesField("id", new BytesRef(doc[0]))); document.add(new NumericDocValuesField("double", Double.doubleToRawLongBits(Double.parseDouble(doc[1])))); - document.add(new NumericDocValuesField("float", Float.floatToRawIntBits(Float.valueOf(doc[2])))); - document.add(new NumericDocValuesField("int", Integer.valueOf(doc[3]))); - document.add(new NumericDocValuesField("long", Long.valueOf(doc[4]))); + document.add(new NumericDocValuesField("float", Float.floatToRawIntBits(Float.parseFloat(doc[2])))); + document.add(new NumericDocValuesField("int", Integer.parseInt(doc[3]))); + document.add(new NumericDocValuesField("long", Long.parseLong(doc[4]))); document.add(new StringField("string", doc[5], Field.Store.NO)); document.add(new SortedDocValuesField("string", new BytesRef(doc[5]))); document.add(new TextField("text", doc[6], Field.Store.NO)); diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java index 3cfa7d0b8bf..cff9efa698e 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java @@ -837,7 +837,7 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer Query q; float fms = fuzzyMinSim; try { - fms = Float.valueOf(fuzzySlop.image.substring(1)).floatValue(); + fms = Float.parseFloat(fuzzySlop.image.substring(1)); } catch (Exception ignored) { } if(fms < 0.0f){ throw new ParseException("Minimum similarity for a FuzzyQuery has to be between 0.0f and 1.0f !"); @@ -853,7 +853,7 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer int s = phraseSlop; // default if (fuzzySlop != null) { try { - s = Float.valueOf(fuzzySlop.image.substring(1)).intValue(); + s = (int)Float.parseFloat(fuzzySlop.image.substring(1)); } catch (Exception ignored) { } } @@ -865,7 +865,7 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer if (boost != null) { float f = (float) 1.0; try { - f = Float.valueOf(boost.image).floatValue(); + f = Float.parseFloat(boost.image); } catch (Exception ignored) { /* Should this be handled somehow? (defaults to "no boost", if diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java index ed0d67c84d9..8ba34a6ff4e 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java @@ -466,7 +466,7 @@ public class StandardSyntaxParser implements SyntaxParser, StandardSyntaxParserC if (boost != null) { float f = (float)1.0; try { - f = Float.valueOf(boost.image).floatValue(); + f = Float.parseFloat(boost.image); // avoid boosting null queries, such as those caused by stop words if (q != null) { q = new BoostQueryNode(q, f); @@ -542,7 +542,7 @@ public class StandardSyntaxParser implements SyntaxParser, StandardSyntaxParserC if (fuzzy) { float fms = defaultMinSimilarity; try { - fms = Float.valueOf(fuzzySlop.image.substring(1)).floatValue(); + fms = Float.parseFloat(fuzzySlop.image.substring(1)); } catch (Exception ignored) { } if(fms < 0.0f){ {if (true) throw new ParseException(new MessageImpl(QueryParserMessages.INVALID_SYNTAX_FUZZY_LIMITS));} @@ -661,7 +661,7 @@ public class StandardSyntaxParser implements SyntaxParser, StandardSyntaxParserC if (fuzzySlop != null) { try { - phraseSlop = Float.valueOf(fuzzySlop.image.substring(1)).intValue(); + phraseSlop = (int)Float.parseFloat(fuzzySlop.image.substring(1)); q = new SlopQueryNode(q, phraseSlop); } catch (Exception ignored) { @@ -679,7 +679,7 @@ public class StandardSyntaxParser implements SyntaxParser, StandardSyntaxParserC if (boost != null) { float f = (float)1.0; try { - f = Float.valueOf(boost.image).floatValue(); + f = Float.parseFloat(boost.image); // avoid boosting null queries, such as those caused by stop words if (q != null) { q = new BoostQueryNode(q, f); diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj index 868b257b636..b53bab3c674 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj @@ -391,7 +391,7 @@ QueryNode Clause(CharSequence field) : { if (boost != null) { float f = (float)1.0; try { - f = Float.valueOf(boost.image).floatValue(); + f = Float.parseFloat(boost.image); // avoid boosting null queries, such as those caused by stop words if (q != null) { q = new BoostQueryNode(q, f); @@ -431,7 +431,7 @@ QueryNode Term(CharSequence field) : { if (fuzzy) { float fms = defaultMinSimilarity; try { - fms = Float.valueOf(fuzzySlop.image.substring(1)).floatValue(); + fms = Float.parseFloat(fuzzySlop.image.substring(1)); } catch (Exception ignored) { } if(fms < 0.0f){ throw new ParseException(new MessageImpl(QueryParserMessages.INVALID_SYNTAX_FUZZY_LIMITS)); @@ -472,7 +472,7 @@ QueryNode Term(CharSequence field) : { if (fuzzySlop != null) { try { - phraseSlop = Float.valueOf(fuzzySlop.image.substring(1)).intValue(); + phraseSlop = (int)Float.parseFloat(fuzzySlop.image.substring(1)); q = new SlopQueryNode(q, phraseSlop); } catch (Exception ignored) { @@ -488,7 +488,7 @@ QueryNode Term(CharSequence field) : { if (boost != null) { float f = (float)1.0; try { - f = Float.valueOf(boost.image).floatValue(); + f = Float.parseFloat(boost.image); // avoid boosting null queries, such as those caused by stop words if (q != null) { q = new BoostQueryNode(q, f); diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java index bd91f045dc9..f0f4b34f8f1 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java @@ -481,7 +481,7 @@ public class QueryParser implements QueryParserConstants { weight = jj_consume_token(NUMBER); float f; try { - f = Float.valueOf(weight.image).floatValue(); + f = Float.parseFloat(weight.image); } catch (Exception floatExc) { {if (true) throw new ParseException(boostErrorMessage + weight.image + " (" + floatExc + ")");} } diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.jj b/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.jj index d63189d73aa..857cca16df7 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.jj +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.jj @@ -460,7 +460,7 @@ void OptionalWeights(SrndQuery q) : { ( weight= { float f; try { - f = Float.valueOf(weight.image).floatValue(); + f = Float.parseFloat(weight.image); } catch (Exception floatExc) { throw new ParseException(boostErrorMessage + weight.image + " (" + floatExc + ")"); } diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/PointRangeQueryBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/PointRangeQueryBuilder.java index 82f7039f0be..1297ad48843 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/PointRangeQueryBuilder.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/PointRangeQueryBuilder.java @@ -79,20 +79,20 @@ public class PointRangeQueryBuilder implements QueryBuilder { try { if (type.equalsIgnoreCase("int")) { return IntPoint.newRangeQuery(field, - (lowerTerm == null ? Integer.MIN_VALUE : Integer.valueOf(lowerTerm)), - (upperTerm == null ? Integer.MAX_VALUE : Integer.valueOf(upperTerm))); + (lowerTerm == null ? Integer.MIN_VALUE : Integer.parseInt(lowerTerm)), + (upperTerm == null ? Integer.MAX_VALUE : Integer.parseInt(upperTerm))); } else if (type.equalsIgnoreCase("long")) { return LongPoint.newRangeQuery(field, - (lowerTerm == null ? Long.MIN_VALUE : Long.valueOf(lowerTerm)), - (upperTerm == null ? Long.MAX_VALUE : Long.valueOf(upperTerm))); + (lowerTerm == null ? Long.MIN_VALUE : Long.parseLong(lowerTerm)), + (upperTerm == null ? Long.MAX_VALUE : Long.parseLong(upperTerm))); } else if (type.equalsIgnoreCase("double")) { return DoublePoint.newRangeQuery(field, - (lowerTerm == null ? Double.NEGATIVE_INFINITY : Double.valueOf(lowerTerm)), - (upperTerm == null ? Double.POSITIVE_INFINITY : Double.valueOf(upperTerm))); + (lowerTerm == null ? Double.NEGATIVE_INFINITY : Double.parseDouble(lowerTerm)), + (upperTerm == null ? Double.POSITIVE_INFINITY : Double.parseDouble(upperTerm))); } else if (type.equalsIgnoreCase("float")) { return FloatPoint.newRangeQuery(field, - (lowerTerm == null ? Float.NEGATIVE_INFINITY : Float.valueOf(lowerTerm)), - (upperTerm == null ? Float.POSITIVE_INFINITY : Float.valueOf(upperTerm))); + (lowerTerm == null ? Float.NEGATIVE_INFINITY : Float.parseFloat(lowerTerm)), + (upperTerm == null ? Float.POSITIVE_INFINITY : Float.parseFloat(upperTerm))); } else { throw new ParserException("type attribute must be one of: [long, int, double, float]"); } diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java index e8533e0a11d..34507941d58 100644 --- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java +++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java @@ -193,7 +193,7 @@ public class TestQueryParser extends QueryParserTestBase { if(fuzzySlop.image.endsWith("€")) { float fms = fuzzyMinSim; try { - fms = Float.valueOf(fuzzySlop.image.substring(1, fuzzySlop.image.length()-1)).floatValue(); + fms = Float.parseFloat(fuzzySlop.image.substring(1, fuzzySlop.image.length()-1)); } catch (Exception ignored) { } float value = Float.parseFloat(termImage); return getRangeQuery(qfield, Float.toString(value-fms/2.f), Float.toString(value+fms/2.f), true, true); diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java index 5fa1523adfc..475688875bc 100644 --- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java +++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java @@ -52,7 +52,7 @@ class CoreParserTestIndexData implements Closeable { Document doc = new Document(); doc.add(LuceneTestCase.newTextField("date", date, Field.Store.YES)); doc.add(LuceneTestCase.newTextField("contents", content, Field.Store.YES)); - doc.add(new IntPoint("date3", Integer.valueOf(date))); + doc.add(new IntPoint("date3", Integer.parseInt(date))); writer.addDocument(doc); line = d.readLine(); } diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/RangeEndpointCalculator.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/RangeEndpointCalculator.java index c3c2088de81..fa290228bbc 100644 --- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/RangeEndpointCalculator.java +++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/RangeEndpointCalculator.java @@ -265,7 +265,7 @@ public abstract class RangeEndpointCalculator> { @Override public Float parseAndAddGap(Float value, String gap) { - return new Float(value.floatValue() + Float.valueOf(gap).floatValue()); + return new Float(value.floatValue() + Float.parseFloat(gap)); } } @@ -281,7 +281,7 @@ public abstract class RangeEndpointCalculator> { @Override public Double parseAndAddGap(Double value, String gap) { - return new Double(value.doubleValue() + Double.valueOf(gap).doubleValue()); + return new Double(value.doubleValue() + Double.parseDouble(gap)); } } @@ -297,7 +297,7 @@ public abstract class RangeEndpointCalculator> { @Override public Integer parseAndAddGap(Integer value, String gap) { - return new Integer(value.intValue() + Integer.valueOf(gap).intValue()); + return new Integer(value.intValue() + Integer.parseInt(gap)); } } @@ -313,7 +313,7 @@ public abstract class RangeEndpointCalculator> { @Override public Long parseAndAddGap(Long value, String gap) { - return new Long(value.longValue() + Long.valueOf(gap).longValue()); + return new Long(value.longValue() + Long.parseLong(gap)); } } diff --git a/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java b/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java index 0258c33022c..7545eacb18c 100644 --- a/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java +++ b/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java @@ -843,7 +843,7 @@ public class MailEntityProcessor extends EntityProcessorBase { String val = context.getEntityAttribute(prop); if (val != null) { val = context.replaceTokens(val); - v = Integer.valueOf(val); + v = Integer.parseInt(val); } } catch (NumberFormatException e) { // do nothing diff --git a/solr/contrib/velocity/src/java/org/apache/solr/response/PageTool.java b/solr/contrib/velocity/src/java/org/apache/solr/response/PageTool.java index 1947f36167a..48dc82619e8 100644 --- a/solr/contrib/velocity/src/java/org/apache/solr/response/PageTool.java +++ b/solr/contrib/velocity/src/java/org/apache/solr/response/PageTool.java @@ -38,7 +38,7 @@ public class PageTool { String rows = request.getParams().get("rows"); if (rows != null) { - results_per_page = new Integer(rows); + results_per_page = Integer.parseInt(rows); } //TODO: Handle group by results Object docs = response.getResponse(); diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java index a07496fcdf1..33e80913e68 100644 --- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java +++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java @@ -697,7 +697,7 @@ public class IndexFetcher { int indexCount = 1, confFilesCount = 1; if (props.containsKey(TIMES_INDEX_REPLICATED)) { - indexCount = Integer.valueOf(props.getProperty(TIMES_INDEX_REPLICATED)) + 1; + indexCount = Integer.parseInt(props.getProperty(TIMES_INDEX_REPLICATED)) + 1; } StringBuilder sb = readToStringBuilder(replicationTime, props.getProperty(INDEX_REPLICATED_AT_LIST)); props.setProperty(INDEX_REPLICATED_AT_LIST, sb.toString()); @@ -708,7 +708,7 @@ public class IndexFetcher { props.setProperty(CONF_FILES_REPLICATED, confFiles.toString()); props.setProperty(CONF_FILES_REPLICATED_AT, String.valueOf(replicationTime)); if (props.containsKey(TIMES_CONFIG_REPLICATED)) { - confFilesCount = Integer.valueOf(props.getProperty(TIMES_CONFIG_REPLICATED)) + 1; + confFilesCount = Integer.parseInt(props.getProperty(TIMES_CONFIG_REPLICATED)) + 1; } props.setProperty(TIMES_CONFIG_REPLICATED, String.valueOf(confFilesCount)); } @@ -717,7 +717,7 @@ public class IndexFetcher { if (!successfulInstall) { int numFailures = 1; if (props.containsKey(TIMES_FAILED)) { - numFailures = Integer.valueOf(props.getProperty(TIMES_FAILED)) + 1; + numFailures = Integer.parseInt(props.getProperty(TIMES_FAILED)) + 1; } props.setProperty(TIMES_FAILED, String.valueOf(numFailures)); props.setProperty(REPLICATION_FAILED_AT, String.valueOf(replicationTime)); diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java index e40b2c38311..4f6a408d533 100644 --- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java @@ -1075,7 +1075,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw String ss[] = s.split(","); List l = new ArrayList<>(); for (String s1 : ss) { - l.add(new Date(Long.valueOf(s1)).toString()); + l.add(new Date(Long.parseLong(s1)).toString()); } nl.add(key, l); } else { diff --git a/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java b/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java index 3ac73008471..c2348669b90 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java +++ b/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java @@ -659,7 +659,7 @@ public class RangeFacetRequest extends FacetComponent.FacetBase { @Override public Float parseAndAddGap(Float value, String gap) { - return new Float(value.floatValue() + Float.valueOf(gap).floatValue()); + return new Float(value.floatValue() + Float.parseFloat(gap)); } } @@ -677,7 +677,7 @@ public class RangeFacetRequest extends FacetComponent.FacetBase { @Override public Double parseAndAddGap(Double value, String gap) { - return new Double(value.doubleValue() + Double.valueOf(gap).doubleValue()); + return new Double(value.doubleValue() + Double.parseDouble(gap)); } } @@ -695,7 +695,7 @@ public class RangeFacetRequest extends FacetComponent.FacetBase { @Override public Integer parseAndAddGap(Integer value, String gap) { - return new Integer(value.intValue() + Integer.valueOf(gap).intValue()); + return new Integer(value.intValue() + Integer.parseInt(gap)); } } @@ -713,7 +713,7 @@ public class RangeFacetRequest extends FacetComponent.FacetBase { @Override public Long parseAndAddGap(Long value, String gap) { - return new Long(value.longValue() + Long.valueOf(gap).longValue()); + return new Long(value.longValue() + Long.parseLong(gap)); } } diff --git a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java index 84ffcb94568..cb3b1eedf05 100644 --- a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java +++ b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java @@ -623,7 +623,7 @@ public abstract class SolrQueryParserBase extends QueryBuilder { } else if (fuzzy) { float fms = fuzzyMinSim; try { - fms = Float.valueOf(fuzzySlop.image.substring(1)).floatValue(); + fms = Float.parseFloat(fuzzySlop.image.substring(1)); } catch (Exception ignored) { } if(fms < 0.0f){ throw new SyntaxError("Minimum similarity for a FuzzyQuery has to be between 0.0f and 1.0f !"); @@ -644,7 +644,7 @@ public abstract class SolrQueryParserBase extends QueryBuilder { int s = phraseSlop; // default if (fuzzySlop != null) { try { - s = Float.valueOf(fuzzySlop.image.substring(1)).intValue(); + s = (int)Float.parseFloat(fuzzySlop.image.substring(1)); } catch (Exception ignored) { } } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java index 276af5fffed..a50fa2c9811 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRange.java @@ -499,7 +499,7 @@ class FacetRangeProcessor extends FacetProcessor { } @Override public Float parseAndAddGap(Comparable value, String gap) { - return new Float(((Number)value).floatValue() + Float.valueOf(gap).floatValue()); + return new Float(((Number)value).floatValue() + Float.parseFloat(gap)); } } private static class DoubleCalc extends Calc { @@ -520,7 +520,7 @@ class FacetRangeProcessor extends FacetProcessor { } @Override public Double parseAndAddGap(Comparable value, String gap) { - return new Double(((Number)value).doubleValue() + Double.valueOf(gap).doubleValue()); + return new Double(((Number)value).doubleValue() + Double.parseDouble(gap)); } } private static class IntCalc extends Calc { @@ -532,7 +532,7 @@ class FacetRangeProcessor extends FacetProcessor { } @Override public Integer parseAndAddGap(Comparable value, String gap) { - return new Integer(((Number)value).intValue() + Integer.valueOf(gap).intValue()); + return new Integer(((Number)value).intValue() + Integer.parseInt(gap)); } } private static class LongCalc extends Calc { @@ -544,7 +544,7 @@ class FacetRangeProcessor extends FacetProcessor { } @Override public Long parseAndAddGap(Comparable value, String gap) { - return new Long(((Number)value).longValue() + Long.valueOf(gap).longValue()); + return new Long(((Number)value).longValue() + Long.parseLong(gap)); } } private static class DateCalc extends Calc { diff --git a/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java index b642d8993ff..8ee5ff25f9e 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java +++ b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java @@ -99,7 +99,7 @@ public class TolerantUpdateProcessorFactory extends UpdateRequestProcessorFactor Object maxErrorsObj = args.get(MAX_ERRORS_PARAM); if (maxErrorsObj != null) { try { - defaultMaxErrors = Integer.valueOf(maxErrorsObj.toString()); + defaultMaxErrors = Integer.parseInt(maxErrorsObj.toString()); } catch (Exception e) { throw new SolrException(ErrorCode.SERVER_ERROR, "Unnable to parse maxErrors parameter: " + maxErrorsObj, e); } diff --git a/solr/core/src/java/org/apache/solr/util/DateMathParser.java b/solr/core/src/java/org/apache/solr/util/DateMathParser.java index 643fde8f15a..2124d1d189c 100644 --- a/solr/core/src/java/org/apache/solr/util/DateMathParser.java +++ b/solr/core/src/java/org/apache/solr/util/DateMathParser.java @@ -381,7 +381,7 @@ public class DateMathParser { } int val = 0; try { - val = Integer.valueOf(ops[pos++]); + val = Integer.parseInt(ops[pos++]); } catch (NumberFormatException e) { throw new ParseException ("Not a Number: \"" + ops[pos-1] + "\"", pos-1); diff --git a/solr/core/src/java/org/apache/solr/util/SolrPluginUtils.java b/solr/core/src/java/org/apache/solr/util/SolrPluginUtils.java index 93866005718..4445e07e1b0 100644 --- a/solr/core/src/java/org/apache/solr/util/SolrPluginUtils.java +++ b/solr/core/src/java/org/apache/solr/util/SolrPluginUtils.java @@ -583,8 +583,8 @@ public class SolrPluginUtils { String[] fieldAndSlopVsBoost = caratPattern.split(s); String[] fieldVsSlop = tildePattern.split(fieldAndSlopVsBoost[0]); String field = fieldVsSlop[0]; - int slop = (2 == fieldVsSlop.length) ? Integer.valueOf(fieldVsSlop[1]) : defaultSlop; - Float boost = (1 == fieldAndSlopVsBoost.length) ? 1 : Float.valueOf(fieldAndSlopVsBoost[1]); + int slop = (2 == fieldVsSlop.length) ? Integer.parseInt(fieldVsSlop[1]) : defaultSlop; + float boost = (1 == fieldAndSlopVsBoost.length) ? 1 : Float.parseFloat(fieldAndSlopVsBoost[1]); FieldParams fp = new FieldParams(field,wordGrams,slop,boost); out.add(fp); } diff --git a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java index bb56a942ecf..9503ee45a2f 100644 --- a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java +++ b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java @@ -295,7 +295,7 @@ public class TestSolrCloudSnapshots extends SolrCloudTestCase { for(int i = 0 ; i < apiResult.size(); i++) { String commitName = apiResult.getName(i); String indexDirPath = (String)((NamedList)apiResult.get(commitName)).get(SolrSnapshotManager.INDEX_DIR_PATH); - long genNumber = Long.valueOf((String)((NamedList)apiResult.get(commitName)).get(SolrSnapshotManager.GENERATION_NUM)); + long genNumber = Long.parseLong((String)((NamedList)apiResult.get(commitName)).get(SolrSnapshotManager.GENERATION_NUM)); result.add(new SnapshotMetaData(commitName, indexDirPath, genNumber)); } return result; diff --git a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java index da6dbac6231..7a9b0bb6c5f 100644 --- a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java +++ b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java @@ -293,7 +293,7 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase { for(int i = 0 ; i < apiResult.size(); i++) { String commitName = apiResult.getName(i); String indexDirPath = (String)((NamedList)apiResult.get(commitName)).get("indexDirPath"); - long genNumber = Long.valueOf((String)((NamedList)apiResult.get(commitName)).get("generation")); + long genNumber = Long.parseLong((String)((NamedList)apiResult.get(commitName)).get("generation")); result.add(new SnapshotMetaData(commitName, indexDirPath, genNumber)); } return result; diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java b/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java index 35bdec643bc..d11c9192090 100644 --- a/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java +++ b/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java @@ -68,7 +68,7 @@ public class TestSolrFieldCacheMBean extends SolrTestCaseJ4 { private void assertEntryListIncluded(boolean checkJmx) { SolrFieldCacheMBean mbean = new SolrFieldCacheMBean(); NamedList stats = checkJmx ? mbean.getStatisticsForJmx() : mbean.getStatistics(); - assert(new Integer(stats.get("entries_count").toString()) > 0); + assert(Integer.parseInt(stats.get("entries_count").toString()) > 0); assertNotNull(stats.get("total_size")); assertNotNull(stats.get("entry#0")); } @@ -76,7 +76,7 @@ public class TestSolrFieldCacheMBean extends SolrTestCaseJ4 { private void assertEntryListNotIncluded(boolean checkJmx) { SolrFieldCacheMBean mbean = new SolrFieldCacheMBean(); NamedList stats = checkJmx ? mbean.getStatisticsForJmx() : mbean.getStatistics(); - assert(new Integer(stats.get("entries_count").toString()) > 0); + assert(Integer.parseInt(stats.get("entries_count").toString()) > 0); assertNull(stats.get("total_size")); assertNull(stats.get("entry#0")); } diff --git a/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java b/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java index e3a8d7b2d64..f502f24ba2e 100644 --- a/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java +++ b/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java @@ -102,7 +102,7 @@ public class CloudMLTQParserTest extends SolrCloudTestCase { int[] actualIds = new int[10]; int i = 0; for (SolrDocument solrDocument : solrDocuments) { - actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id"))); + actualIds[i++] = Integer.parseInt(String.valueOf(solrDocument.getFieldValue("id"))); } assertArrayEquals(expectedIds, actualIds); @@ -117,7 +117,7 @@ public class CloudMLTQParserTest extends SolrCloudTestCase { int[] actualIds = new int[solrDocuments.size()]; int i = 0; for (SolrDocument solrDocument : solrDocuments) { - actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id"))); + actualIds[i++] = Integer.parseInt(String.valueOf(solrDocument.getFieldValue("id"))); } assertArrayEquals(expectedIds, actualIds); @@ -127,7 +127,7 @@ public class CloudMLTQParserTest extends SolrCloudTestCase { actualIds = new int[solrDocuments.size()]; i = 0; for (SolrDocument solrDocument : solrDocuments) { - actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id"))); + actualIds[i++] = Integer.parseInt(String.valueOf(solrDocument.getFieldValue("id"))); } System.out.println("DEBUG ACTUAL IDS 1: " + Arrays.toString(actualIds)); assertArrayEquals(expectedIds, actualIds); @@ -138,7 +138,7 @@ public class CloudMLTQParserTest extends SolrCloudTestCase { actualIds = new int[solrDocuments.size()]; i = 0; for (SolrDocument solrDocument : solrDocuments) { - actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id"))); + actualIds[i++] = Integer.parseInt(String.valueOf(solrDocument.getFieldValue("id"))); } System.out.println("DEBUG ACTUAL IDS 2: " + Arrays.toString(actualIds)); assertArrayEquals(expectedIds, actualIds); @@ -154,7 +154,7 @@ public class CloudMLTQParserTest extends SolrCloudTestCase { int[] actualIds = new int[solrDocuments.size()]; int i = 0; for (SolrDocument solrDocument : solrDocuments) { - actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id"))); + actualIds[i++] = Integer.parseInt(String.valueOf(solrDocument.getFieldValue("id"))); } assertArrayEquals(expectedIds, actualIds); @@ -184,7 +184,7 @@ public class CloudMLTQParserTest extends SolrCloudTestCase { int[] actualIds = new int[solrDocuments.size()]; int i = 0; for (SolrDocument solrDocument : solrDocuments) { - actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id"))); + actualIds[i++] = Integer.parseInt(String.valueOf(solrDocument.getFieldValue("id"))); } assertArrayEquals(expectedIds, actualIds); @@ -236,7 +236,7 @@ public class CloudMLTQParserTest extends SolrCloudTestCase { int i = 0; StringBuilder sb = new StringBuilder(); for (SolrDocument solrDocument : solrDocuments) { - actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id"))); + actualIds[i++] = Integer.parseInt(String.valueOf(solrDocument.getFieldValue("id"))); sb.append(actualIds[i-1]).append(", "); } assertArrayEquals(expectedIds, actualIds); From cce7ba9b2f1849930fc83cb5ede580f26c6d31f6 Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Wed, 15 Mar 2017 17:25:06 +0530 Subject: [PATCH 14/20] SOLR-9516: Fix: Admin UI (angular) didn't work with Kerberos --- solr/CHANGES.txt | 2 ++ solr/webapp/web/WEB-INF/web.xml | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index b3ce30cbf08..0f1f488d890 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -253,6 +253,8 @@ Bug Fixes * SOLR-10184: Fix bin/solr so it can run properly on java9 (hossman, Uwe Schindler) +* SOLR-9516: Admin UI (angular) didn't work with Kerberos (Cassandra Targett, Amrit Sarkar via Ishan Chattopadhyaya) + Optimizations ---------------------- diff --git a/solr/webapp/web/WEB-INF/web.xml b/solr/webapp/web/WEB-INF/web.xml index 5278ae58c59..5ebce106b32 100644 --- a/solr/webapp/web/WEB-INF/web.xml +++ b/solr/webapp/web/WEB-INF/web.xml @@ -56,7 +56,7 @@ --> excludePatterns - /css/.+,/js/.+,/img/.+,/tpl/.+ + /libs/.+,/css/.+,/js/.+,/img/.+,/tpl/.+ From 0170df93f3665c941cdd1b7c75d139bc1ed7af96 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Wed, 15 Mar 2017 10:31:10 +0000 Subject: [PATCH 15/20] SOLR-10046: Add UninvertDocValuesMergePolicyFactory class. (Keith Laban, Christine Poerschke) --- solr/CHANGES.txt | 2 + .../UninvertDocValuesMergePolicyFactory.java | 218 ++++++++++++++++ .../collection1/conf/schema-docValues.xml | 1 + ...ig-uninvertdocvaluesmergepolicyfactory.xml | 38 +++ .../UninvertDocValuesMergePolicyTest.java | 243 ++++++++++++++++++ 5 files changed, 502 insertions(+) create mode 100644 solr/core/src/java/org/apache/solr/index/UninvertDocValuesMergePolicyFactory.java create mode 100644 solr/core/src/test-files/solr/collection1/conf/solrconfig-uninvertdocvaluesmergepolicyfactory.xml create mode 100644 solr/core/src/test/org/apache/solr/index/UninvertDocValuesMergePolicyTest.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 0f1f488d890..6829cd1952e 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -185,6 +185,8 @@ New Features * SOLR-10224: Add disk total and disk free metrics. (ab) +* SOLR-10046: Add UninvertDocValuesMergePolicyFactory class. (Keith Laban, Christine Poerschke) + Bug Fixes ---------------------- diff --git a/solr/core/src/java/org/apache/solr/index/UninvertDocValuesMergePolicyFactory.java b/solr/core/src/java/org/apache/solr/index/UninvertDocValuesMergePolicyFactory.java new file mode 100644 index 00000000000..b6bfbedcf9c --- /dev/null +++ b/solr/core/src/java/org/apache/solr/index/UninvertDocValuesMergePolicyFactory.java @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.solr.index; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CodecReader; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.FilterCodecReader; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.OneMergeWrappingMergePolicy; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.solr.core.SolrResourceLoader; +import org.apache.solr.schema.IndexSchema; +import org.apache.solr.schema.SchemaField; +import org.apache.solr.uninverting.UninvertingReader; + +/** + * A merge policy that can detect schema changes and write docvalues into merging segments when a field has docvalues enabled + * Using UninvertingReader. + * + * This merge policy will delegate to the wrapped merge policy for selecting merge segments + * + */ +public class UninvertDocValuesMergePolicyFactory extends WrapperMergePolicyFactory { + + final private boolean skipIntegrityCheck; + + /** + * Whether or not the wrapped docValues producer should check consistency + */ + public boolean getSkipIntegrityCheck() { + return skipIntegrityCheck; + } + + public UninvertDocValuesMergePolicyFactory(SolrResourceLoader resourceLoader, MergePolicyFactoryArgs args, IndexSchema schema) { + super(resourceLoader, args, schema); + final Boolean sic = (Boolean)args.remove("skipIntegrityCheck"); + if (sic != null) { + this.skipIntegrityCheck = sic.booleanValue(); + } else { + this.skipIntegrityCheck = false; + } + if (!args.keys().isEmpty()) { + throw new IllegalArgumentException("Arguments were "+args+" but "+getClass().getSimpleName()+" takes no arguments."); + } + } + + @Override + protected MergePolicy getMergePolicyInstance(MergePolicy wrappedMP) { + return new OneMergeWrappingMergePolicy(wrappedMP, (merge) -> new UninvertDocValuesOneMerge(merge.segments)); + } + + private UninvertingReader.Type getUninversionType(FieldInfo fi) { + SchemaField sf = schema.getFieldOrNull(fi.name); + + if (null != sf && + sf.hasDocValues() && + fi.getDocValuesType() == DocValuesType.NONE && + fi.getIndexOptions() != IndexOptions.NONE) { + return sf.getType().getUninversionType(sf); + } else { + return null; + } + } + + private class UninvertDocValuesOneMerge extends MergePolicy.OneMerge { + + public UninvertDocValuesOneMerge(List segments) { + super(segments); + } + + @Override + public CodecReader wrapForMerge(CodecReader reader) throws IOException { + // Wrap the reader with an uninverting reader if any of the fields have no docvalues but the + // Schema says there should be + + + Map uninversionMap = null; + + for(FieldInfo fi: reader.getFieldInfos()) { + final UninvertingReader.Type type = getUninversionType(fi); + if (type != null) { + if (uninversionMap == null) { + uninversionMap = new HashMap<>(); + } + uninversionMap.put(fi.name, type); + } + + } + + if(uninversionMap == null) { + return reader; // Default to normal reader if nothing to uninvert + } else { + return new UninvertingFilterCodecReader(reader, uninversionMap); + } + + } + + } + + + /** + * Delegates to an Uninverting for fields with docvalues + * + * This is going to blow up FieldCache, look into an alternative implementation that uninverts without + * fieldcache + */ + private class UninvertingFilterCodecReader extends FilterCodecReader { + + private final UninvertingReader uninvertingReader; + private final DocValuesProducer docValuesProducer; + + public UninvertingFilterCodecReader(CodecReader in, Map uninversionMap) { + super(in); + + this.uninvertingReader = new UninvertingReader(in, uninversionMap); + this.docValuesProducer = new DocValuesProducer() { + + @Override + public NumericDocValues getNumeric(FieldInfo field) throws IOException { + return uninvertingReader.getNumericDocValues(field.name); + } + + @Override + public BinaryDocValues getBinary(FieldInfo field) throws IOException { + return uninvertingReader.getBinaryDocValues(field.name); + } + + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + return uninvertingReader.getSortedDocValues(field.name); + } + + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return uninvertingReader.getSortedNumericDocValues(field.name); + } + + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + return uninvertingReader.getSortedSetDocValues(field.name); + } + + @Override + public void checkIntegrity() throws IOException { + if (!skipIntegrityCheck) { + uninvertingReader.checkIntegrity(); + } + } + + @Override + public void close() throws IOException { + } + + @Override + public long ramBytesUsed() { + return 0; + } + }; + } + + @Override + protected void doClose() throws IOException { + docValuesProducer.close(); + uninvertingReader.close(); + super.doClose(); + } + + @Override + public DocValuesProducer getDocValuesReader() { + return docValuesProducer; + } + + @Override + public FieldInfos getFieldInfos() { + return uninvertingReader.getFieldInfos(); + } + + @Override + public CacheHelper getCoreCacheHelper() { + return in.getCoreCacheHelper(); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return in.getReaderCacheHelper(); + } + + } + +} diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml index c7b7de8c6b3..9e4286d226b 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValues.xml @@ -62,6 +62,7 @@ + diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-uninvertdocvaluesmergepolicyfactory.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-uninvertdocvaluesmergepolicyfactory.xml new file mode 100644 index 00000000000..613357bf91b --- /dev/null +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-uninvertdocvaluesmergepolicyfactory.xml @@ -0,0 +1,38 @@ + + + + + + ${tests.luceneMatchVersion:LATEST} + + + + + ${useCompoundFile:false} + + inner + org.apache.solr.index.DefaultMergePolicyFactory + ${solr.tests.skipIntegrityCheck:false} + + + + + + + + diff --git a/solr/core/src/test/org/apache/solr/index/UninvertDocValuesMergePolicyTest.java b/solr/core/src/test/org/apache/solr/index/UninvertDocValuesMergePolicyTest.java new file mode 100644 index 00000000000..17e6b3e4191 --- /dev/null +++ b/solr/core/src/test/org/apache/solr/index/UninvertDocValuesMergePolicyTest.java @@ -0,0 +1,243 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.solr.index; + +import java.util.Random; +import java.util.function.IntUnaryOperator; + +import org.apache.lucene.document.Document; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.SortedDocValues; +import org.apache.solr.SolrTestCaseJ4; +import org.apache.solr.core.SolrCore; +import org.apache.solr.schema.IndexSchema; +import org.apache.solr.schema.SchemaField; +import org.apache.solr.search.SolrIndexSearcher; +import org.apache.solr.util.RefCounted; +import org.apache.solr.util.TestHarness; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +public class UninvertDocValuesMergePolicyTest extends SolrTestCaseJ4 { + + private static String SOLR_TESTS_SKIP_INTEGRITY_CHECK = "solr.tests.skipIntegrityCheck"; + private static String ID_FIELD = "id"; + private static String TEST_FIELD = "string_add_dv_later"; + + @BeforeClass + public static void beforeTests() throws Exception { + System.setProperty(SOLR_TESTS_SKIP_INTEGRITY_CHECK, (random().nextBoolean() ? "true" : "false")); + } + + @AfterClass + public static void afterTests() { + System.clearProperty(SOLR_TESTS_SKIP_INTEGRITY_CHECK); + } + + @After + public void after() throws Exception { + deleteCore(); + } + + @Before + public void before() throws Exception { + initCore("solrconfig-uninvertdocvaluesmergepolicyfactory.xml", "schema-docValues.xml"); + } + + public void testIndexAndAddDocValues() throws Exception { + Random rand = random(); + + for(int i=0; i < 100; i++) { + assertU(adoc(ID_FIELD, String.valueOf(i), TEST_FIELD, String.valueOf(i))); + + if(rand.nextBoolean()) { + assertU(commit()); + } + } + + assertU(commit()); + + // Assert everything has been indexed and there are no docvalues + withNewRawReader(h, topReader -> { + assertEquals(100, topReader.numDocs()); + + final FieldInfos infos = MultiFields.getMergedFieldInfos(topReader); + + // The global field type should not have docValues yet + assertEquals(DocValuesType.NONE, infos.fieldInfo(TEST_FIELD).getDocValuesType()); + }); + + + addDocValuesTo(h, TEST_FIELD); + + + // Add some more documents with doc values turned on including updating some + for(int i=90; i < 110; i++) { + assertU(adoc(ID_FIELD, String.valueOf(i), TEST_FIELD, String.valueOf(i))); + + if(rand.nextBoolean()) { + assertU(commit()); + } + } + + assertU(commit()); + + withNewRawReader(h, topReader -> { + assertEquals(110, topReader.numDocs()); + + final FieldInfos infos = MultiFields.getMergedFieldInfos(topReader); + // The global field type should have docValues because a document with dvs was added + assertEquals(DocValuesType.SORTED, infos.fieldInfo(TEST_FIELD).getDocValuesType()); + }); + + int optimizeSegments = 1; + assertU(optimize("maxSegments", String.valueOf(optimizeSegments))); + + + // Assert all docs have the right docvalues + withNewRawReader(h, topReader -> { + // Assert merged into one segment + assertEquals(110, topReader.numDocs()); + assertEquals(optimizeSegments, topReader.leaves().size()); + + + final FieldInfos infos = MultiFields.getMergedFieldInfos(topReader); + // The global field type should have docValues because a document with dvs was added + assertEquals(DocValuesType.SORTED, infos.fieldInfo(TEST_FIELD).getDocValuesType()); + + + // Check that all segments have the right docvalues type with the correct value + // Also check that other fields (e.g. the id field) didn't mistakenly get docvalues added + for (LeafReaderContext ctx : topReader.leaves()) { + LeafReader r = ctx.reader(); + SortedDocValues docvalues = r.getSortedDocValues(TEST_FIELD); + for(int i = 0; i < r.numDocs(); ++i) { + Document doc = r.document(i); + String v = doc.getField(TEST_FIELD).stringValue(); + String id = doc.getField(ID_FIELD).stringValue(); + assertEquals(DocValuesType.SORTED, r.getFieldInfos().fieldInfo(TEST_FIELD).getDocValuesType()); + assertEquals(DocValuesType.NONE, r.getFieldInfos().fieldInfo(ID_FIELD).getDocValuesType()); + assertEquals(v, id); + + docvalues.nextDoc(); + assertEquals(v, docvalues.binaryValue().utf8ToString()); + } + } + }); + } + + + // When an non-indexed field gets merged, it exhibit the old behavior + // The field will be merged, docvalues headers updated, but no docvalues for this field + public void testNonIndexedFieldDoesNonFail() throws Exception { + // Remove Indexed from fieldType + removeIndexFrom(h, TEST_FIELD); + + assertU(adoc(ID_FIELD, String.valueOf(1), TEST_FIELD, String.valueOf(1))); + assertU(commit()); + + addDocValuesTo(h, TEST_FIELD); + + assertU(adoc(ID_FIELD, String.valueOf(2), TEST_FIELD, String.valueOf(2))); + assertU(commit()); + + assertU(optimize("maxSegments", "1")); + + withNewRawReader(h, topReader -> { + // Assert merged into one segment + assertEquals(2, topReader.numDocs()); + assertEquals(1, topReader.leaves().size()); + + + final FieldInfos infos = MultiFields.getMergedFieldInfos(topReader); + // The global field type should have docValues because a document with dvs was added + assertEquals(DocValuesType.SORTED, infos.fieldInfo(TEST_FIELD).getDocValuesType()); + + for (LeafReaderContext ctx : topReader.leaves()) { + LeafReader r = ctx.reader(); + SortedDocValues docvalues = r.getSortedDocValues(TEST_FIELD); + for(int i = 0; i < r.numDocs(); ++i) { + Document doc = r.document(i); + String v = doc.getField(TEST_FIELD).stringValue(); + String id = doc.getField(ID_FIELD).stringValue(); + assertEquals(DocValuesType.SORTED, r.getFieldInfos().fieldInfo(TEST_FIELD).getDocValuesType()); + assertEquals(DocValuesType.NONE, r.getFieldInfos().fieldInfo(ID_FIELD).getDocValuesType()); + + + if(id.equals("2")) { + assertTrue(docvalues.advanceExact(i)); + assertEquals(v, docvalues.binaryValue().utf8ToString()); + } else { + assertFalse(docvalues.advanceExact(i)); + } + + } + } + }); + } + + + private static void addDocValuesTo(TestHarness h, String fieldName) { + implUpdateSchemaField(h, fieldName, (p) -> (p | 0x00008000)); // FieldProperties.DOC_VALUES + } + + private static void removeIndexFrom(TestHarness h, String fieldName) { + implUpdateSchemaField(h, fieldName, (p) -> (p ^ 0x00000001)); // FieldProperties.INDEXED + } + + private static void implUpdateSchemaField(TestHarness h, String fieldName, IntUnaryOperator propertiesModifier) { + try (SolrCore core = h.getCoreInc()) { + + // Add docvalues to the field type + IndexSchema schema = core.getLatestSchema(); + SchemaField oldSchemaField = schema.getField(fieldName); + SchemaField newSchemaField = new SchemaField( + fieldName, + oldSchemaField.getType(), + propertiesModifier.applyAsInt(oldSchemaField.getProperties()), + oldSchemaField.getDefaultValue()); + schema.getFields().put(fieldName, newSchemaField); + } + } + + private interface DirectoryReaderConsumer { + public void accept(DirectoryReader consumer) throws Exception; + } + + private static void withNewRawReader(TestHarness h, DirectoryReaderConsumer consumer) { + try (SolrCore core = h.getCoreInc()) { + final RefCounted searcherRef = core.openNewSearcher(true, true); + final SolrIndexSearcher searcher = searcherRef.get(); + try { + try { + consumer.accept(searcher.getRawReader()); + } catch (Exception e) { + fail(e.toString()); + } + } finally { + searcherRef.decref(); + } + } + } +} From 482ec73ae5be7af0be8ec12bcbe86bc8dff6cccc Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Thu, 16 Mar 2017 02:03:09 +0530 Subject: [PATCH 16/20] SOLR-9516: Updating CHANGES.txt entry --- solr/CHANGES.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 6829cd1952e..f1a12ea6fa5 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -255,7 +255,8 @@ Bug Fixes * SOLR-10184: Fix bin/solr so it can run properly on java9 (hossman, Uwe Schindler) -* SOLR-9516: Admin UI (angular) didn't work with Kerberos (Cassandra Targett, Amrit Sarkar via Ishan Chattopadhyaya) +* SOLR-9516: Admin UI (angular) now works with Kerberos, by excluding serving of /solr/libs/* through + SolrDispatchFilter. (Cassandra Targett, Amrit Sarkar via Ishan Chattopadhyaya) Optimizations ---------------------- From 1e97cf82d253f6d6f721889155caec62625a81e9 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Mon, 13 Mar 2017 12:50:17 -0400 Subject: [PATCH 17/20] SOLR-10270: Stop exporting _version_ during GROUP BY aggregations in map_reduce mode --- .../src/java/org/apache/solr/handler/sql/SolrTable.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java index 9375bc0f00c..37ed7d9f949 100644 --- a/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java +++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java @@ -330,24 +330,16 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { private String getFields(Set fieldSet) { StringBuilder buf = new StringBuilder(); - boolean appendVersion = true; for(String field : fieldSet) { if(buf.length() > 0) { buf.append(","); } - if(field.equals("_version_")) { - appendVersion = false; - } buf.append(field); } - if(appendVersion){ - buf.append(",_version_"); - } - return buf.toString(); } From da56db8bf50c2f33ef407ebe18a957e6da64d388 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Mon, 13 Mar 2017 17:10:21 -0400 Subject: [PATCH 18/20] SOLR-10271: SQL aggregations in map_reduce mode should use javabin transport --- solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java index 37ed7d9f949..644ed97449a 100644 --- a/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java +++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java @@ -453,6 +453,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { params.set(CommonParams.FL, fl); params.set(CommonParams.Q, query); + params.set(CommonParams.WT, CommonParams.JAVABIN); //Always use the /export handler for Group By Queries because it requires exporting full result sets. params.set(CommonParams.QT, "/export"); @@ -691,6 +692,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { params.set(CommonParams.FL, fl); params.set(CommonParams.Q, query); + params.set(CommonParams.WT, CommonParams.JAVABIN); //Always use the /export handler for Distinct Queries because it requires exporting full result sets. params.set(CommonParams.QT, "/export"); From 260159a634d96fe5153a54da4db6485295720ecc Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Wed, 15 Mar 2017 18:31:14 -0400 Subject: [PATCH 19/20] SOLR-10085: SQL result set fields should be ordered by the field list --- .../org/apache/solr/handler/SQLHandler.java | 37 ++++++++++++++----- .../apache/solr/handler/TestSQLHandler.java | 31 ++++++++++++++++ .../apache/solr/client/solrj/io/Tuple.java | 21 ++++++++--- 3 files changed, 74 insertions(+), 15 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java index d65ea560ab5..7563fe84ced 100644 --- a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java @@ -34,7 +34,9 @@ import org.apache.solr.client.solrj.io.stream.ExceptionStream; import org.apache.solr.client.solrj.io.stream.JDBCStream; import org.apache.solr.client.solrj.io.stream.TupleStream; import org.apache.solr.common.SolrException; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.params.SolrParams; import org.apache.solr.core.CoreContainer; import org.apache.solr.core.SolrCore; import org.apache.solr.handler.sql.CalciteSolrDriver; @@ -74,6 +76,9 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware, Per public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception { ModifiableSolrParams params = new ModifiableSolrParams(req.getParams()); + params = adjustParams(params); + req.setParams(params); + String sql = params.get("stmt"); // Set defaults for parameters params.set("numWorkers", params.getInt("numWorkers", 1)); @@ -139,6 +144,8 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware, Per private class SqlHandlerStream extends JDBCStream { private final boolean includeMetadata; private boolean firstTuple = true; + List metadataFields = new ArrayList<>(); + Map metadataAliases = new HashMap<>(); SqlHandlerStream(String connectionUrl, String sqlQuery, StreamComparator definedSort, Properties connectionProperties, String driverClassName, boolean includeMetadata) @@ -151,7 +158,7 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware, Per @Override public Tuple read() throws IOException { // Return a metadata tuple as the first tuple and then pass through to the JDBCStream. - if(includeMetadata && firstTuple) { + if(firstTuple) { try { Map fields = new HashMap<>(); @@ -159,8 +166,6 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware, Per ResultSetMetaData resultSetMetaData = resultSet.getMetaData(); - List metadataFields = new ArrayList<>(); - Map metadataAliases = new HashMap<>(); for(int i = 1; i <= resultSetMetaData.getColumnCount(); i++) { String columnName = resultSetMetaData.getColumnName(i); String columnLabel = resultSetMetaData.getColumnLabel(i); @@ -168,16 +173,30 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware, Per metadataAliases.put(columnName, columnLabel); } - fields.put("isMetadata", true); - fields.put("fields", metadataFields); - fields.put("aliases", metadataAliases); - return new Tuple(fields); + if(includeMetadata) { + fields.put("isMetadata", true); + fields.put("fields", metadataFields); + fields.put("aliases", metadataAliases); + return new Tuple(fields); + } } catch (SQLException e) { throw new IOException(e); } - } else { - return super.read(); } + + Tuple tuple = super.read(); + if(!tuple.EOF) { + tuple.fieldNames = metadataFields; + tuple.fieldLabels = metadataAliases; + } + return tuple; } } + + private ModifiableSolrParams adjustParams(SolrParams params) { + ModifiableSolrParams adjustedParams = new ModifiableSolrParams(); + adjustedParams.add(params); + adjustedParams.add(CommonParams.OMIT_HEADER, "true"); + return adjustedParams; + } } diff --git a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java index f222ceed495..cb16f033a24 100644 --- a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java +++ b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java @@ -16,21 +16,30 @@ */ package org.apache.solr.handler; +import java.io.BufferedReader; import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; import java.util.ArrayList; import java.util.List; import java.util.Locale; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.impl.InputStreamResponseParser; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.stream.ExceptionStream; import org.apache.solr.client.solrj.io.stream.SolrStream; import org.apache.solr.client.solrj.io.stream.TupleStream; +import org.apache.solr.client.solrj.request.QueryRequest; import org.apache.solr.cloud.AbstractFullDistribZkTestBase; import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.SolrParams; +import org.apache.solr.common.util.NamedList; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -161,6 +170,9 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getLong("field_i") == 7); assert(tuple.get("str_s").equals("a")); + //Assert field order + assertResponseContains(clients.get(0), sParams, "{\"docs\":[{\"id\":8,\"field_i\":60,\"str_s\":\"c\"}"); + //Test unlimited unsorted result. Should sort on _version_ desc sParams = mapParams(CommonParams.QT, "/sql", "stmt", "select id, field_i, str_s from collection1 where text='XXXX'"); @@ -2362,4 +2374,23 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { return params; } + public void assertResponseContains(SolrClient server, SolrParams requestParams, String json) throws IOException, SolrServerException { + String p = requestParams.get("qt"); + if(p != null) { + ModifiableSolrParams modifiableSolrParams = (ModifiableSolrParams) requestParams; + modifiableSolrParams.remove("qt"); + } + + QueryRequest query = new QueryRequest( requestParams ); + query.setPath(p); + query.setResponseParser(new InputStreamResponseParser("json")); + query.setMethod(SolrRequest.METHOD.POST); + NamedList genericResponse = server.request(query); + InputStream stream = (InputStream)genericResponse.get("stream"); + InputStreamReader reader = new InputStreamReader(stream, "UTF-8"); + BufferedReader bufferedReader = new BufferedReader(reader); + String response = bufferedReader.readLine(); + assertTrue(response.contains(json)); + } + } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Tuple.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Tuple.java index 58d948dd65f..fdf44c98ee6 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Tuple.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Tuple.java @@ -45,6 +45,8 @@ public class Tuple implements Cloneable, MapWriter { public boolean EXCEPTION; public Map fields = new HashMap(); + public List fieldNames; + public Map fieldLabels; public Tuple(Map fields) { if(fields.containsKey("EOF")) { @@ -198,12 +200,19 @@ public class Tuple implements Cloneable, MapWriter { @Override public void writeMap(EntryWriter ew) throws IOException { - fields.forEach((k, v) -> { - try { - ew.put((String)k,v); - } catch (IOException e) { - throw new RuntimeException(e); + if(fieldNames == null) { + fields.forEach((k, v) -> { + try { + ew.put((String) k, v); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } else { + for(String fieldName : fieldNames) { + String label = fieldLabels.get(fieldName); + ew.put(label, fields.get(label)); } - }); + } } } From 1fe4cff22cec810a9035da5ad6dcc1b747ad48c4 Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Thu, 16 Mar 2017 05:32:27 +0530 Subject: [PATCH 20/20] SOLR-10302: Solr's zkcli scripts now able to find the metrics libraries --- solr/CHANGES.txt | 2 ++ solr/server/scripts/cloud-scripts/zkcli.bat | 2 +- solr/server/scripts/cloud-scripts/zkcli.sh | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index f1a12ea6fa5..5e2a518ddd8 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -258,6 +258,8 @@ Bug Fixes * SOLR-9516: Admin UI (angular) now works with Kerberos, by excluding serving of /solr/libs/* through SolrDispatchFilter. (Cassandra Targett, Amrit Sarkar via Ishan Chattopadhyaya) +* SOLR-10302: Solr's zkcli scripts now able to find the metrics libraries, which it couldn't earlier (kiran, Ishan Chattopadhyaya) + Optimizations ---------------------- diff --git a/solr/server/scripts/cloud-scripts/zkcli.bat b/solr/server/scripts/cloud-scripts/zkcli.bat index c3726850cba..c5d7b72948d 100644 --- a/solr/server/scripts/cloud-scripts/zkcli.bat +++ b/solr/server/scripts/cloud-scripts/zkcli.bat @@ -22,4 +22,4 @@ REM -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^ REM -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD "%JVM%" %SOLR_ZK_CREDS_AND_ACLS% %ZKCLI_JVM_FLAGS% -Dlog4j.configuration="%LOG4J_CONFIG%" ^ --classpath "%SDIR%\..\..\solr-webapp\webapp\WEB-INF\lib\*;%SDIR%\..\..\lib\ext\*" org.apache.solr.cloud.ZkCLI %* +-classpath "%SDIR%\..\..\solr-webapp\webapp\WEB-INF\lib\*;%SDIR%\..\..\lib\ext\*;%SDIR%\..\..\lib\*" org.apache.solr.cloud.ZkCLI %* diff --git a/solr/server/scripts/cloud-scripts/zkcli.sh b/solr/server/scripts/cloud-scripts/zkcli.sh index df4326565c0..bd971e9ee41 100755 --- a/solr/server/scripts/cloud-scripts/zkcli.sh +++ b/solr/server/scripts/cloud-scripts/zkcli.sh @@ -22,5 +22,5 @@ fi # -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD" PATH=$JAVA_HOME/bin:$PATH $JVM $SOLR_ZK_CREDS_AND_ACLS $ZKCLI_JVM_FLAGS -Dlog4j.configuration=$log4j_config \ --classpath "$sdir/../../solr-webapp/webapp/WEB-INF/lib/*:$sdir/../../lib/ext/*" org.apache.solr.cloud.ZkCLI ${1+"$@"} +-classpath "$sdir/../../solr-webapp/webapp/WEB-INF/lib/*:$sdir/../../lib/ext/*:$sdir/../../lib/*" org.apache.solr.cloud.ZkCLI ${1+"$@"}