diff --git a/solr/common-build.xml b/solr/common-build.xml index 2a0ad51c175..3dfafdcb8c1 100644 --- a/solr/common-build.xml +++ b/solr/common-build.xml @@ -22,9 +22,9 @@ - + - + @@ -79,17 +79,17 @@ - + - @@ -134,7 +134,7 @@ - + @@ -152,7 +152,7 @@ - @@ -170,16 +170,16 @@ - - - - + @@ -191,7 +191,7 @@ - + @@ -208,7 +208,7 @@ - + @@ -511,7 +511,7 @@ - + @@ -529,7 +529,7 @@ - + @@ -550,14 +550,13 @@ - - + - + diff --git a/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java b/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java index ba94104bba1..7a6c8478657 100644 --- a/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java +++ b/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java @@ -83,7 +83,7 @@ import org.slf4j.MDC; /** * Run solr using jetty - * + * * @since solr 1.3 */ public class JettySolrRunner { @@ -93,7 +93,7 @@ public class JettySolrRunner { private static final int THREAD_POOL_MAX_THREADS = 10000; // NOTE: needs to be larger than SolrHttpClient.threadPoolSweeperMaxIdleTime private static final int THREAD_POOL_MAX_IDLE_TIME_MS = 260000; - + Server server; volatile FilterHolder dispatchFilter; @@ -128,14 +128,14 @@ public class JettySolrRunner { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private AtomicLong nRequests = new AtomicLong(); - + List delays = new ArrayList<>(); public long getTotalRequests() { return nRequests.get(); } - + /** * Introduce a delay of specified milliseconds for the specified request. * @@ -146,7 +146,7 @@ public class JettySolrRunner { public void addDelay(String reason, int count, int delay) { delays.add(new Delay(reason, count, delay)); } - + /** * Remove any delay introduced before. */ @@ -167,14 +167,14 @@ public class JettySolrRunner { @Override public void destroy() { } - + private void executeDelay() { int delayMs = 0; for (Delay delay: delays) { this.log.info("Delaying "+delay.delayValue+", for reason: "+delay.reason); if (delay.counter.decrementAndGet() == 0) { delayMs += delay.delayValue; - } + } } if (delayMs > 0) { @@ -215,7 +215,7 @@ public class JettySolrRunner { public JettySolrRunner(String solrHome, JettyConfig config) { this(solrHome, new Properties(), config); } - + /** * Construct a JettySolrRunner * @@ -244,7 +244,7 @@ public class JettySolrRunner { this.solrHome = solrHome; this.config = config; this.nodeProperties = nodeProperties; - + if (enableProxy) { try { proxy = new SocketProxy(0, config.sslConfig != null && config.sslConfig.isSSLMode()); @@ -256,7 +256,7 @@ public class JettySolrRunner { this.init(this.config.port); } - + private void init(int port) { QueuedThreadPool qtp = new QueuedThreadPool(); @@ -275,7 +275,7 @@ public class JettySolrRunner { // // This means we will use the same truststore, keystore (and keys) for // the server as well as any client actions taken by this JVM in - // talking to that server, but for the purposes of testing that should + // talking to that server, but for the purposes of testing that should // be good enough final SslContextFactory sslcontext = SSLConfig.createContextFactory(config.sslConfig); @@ -382,7 +382,7 @@ public class JettySolrRunner { dispatchFilter.setHeldClass(SolrDispatchFilter.class); dispatchFilter.setInitParameter("excludePatterns", excludePatterns); root.addFilter(dispatchFilter, "*", EnumSet.of(DispatcherType.REQUEST)); - + synchronized (JettySolrRunner.this) { waitOnSolr = true; JettySolrRunner.this.notify(); @@ -400,7 +400,7 @@ public class JettySolrRunner { } chain = injectJettyHandlers(chain); - + GzipHandler gzipHandler = new GzipHandler(); gzipHandler.setHandler(chain); @@ -413,7 +413,7 @@ public class JettySolrRunner { server.setHandler(gzipHandler); } - /** descendants may inject own handler chaining it to the given root + /** descendants may inject own handler chaining it to the given root * and then returning that own one*/ protected HandlerWrapper injectJettyHandlers(HandlerWrapper chain) { return chain; @@ -445,7 +445,7 @@ public class JettySolrRunner { public boolean isRunning() { return server.isRunning() && dispatchFilter != null && dispatchFilter.isRunning(); } - + public boolean isStopped() { return (server.isStopped() && dispatchFilter == null) || (server.isStopped() && dispatchFilter.isStopped() && ((QueuedThreadPool) server.getThreadPool()).isStopped()); @@ -478,12 +478,12 @@ public class JettySolrRunner { // Do not let Jetty/Solr pollute the MDC for this thread Map prevContext = MDC.getCopyOfContextMap(); MDC.clear(); - + log.info("Start Jetty (original configured port={})", this.config.port); - + try { int port = reusePort && jettyPort != -1 ? jettyPort : this.config.port; - + // if started before, make a new server if (startedBefore) { waitOnSolr = false; @@ -508,21 +508,21 @@ public class JettySolrRunner { } } } - + if (config.waitForLoadingCoresToFinishMs != null && config.waitForLoadingCoresToFinishMs > 0L) { waitForLoadingCoresToFinish(config.waitForLoadingCoresToFinishMs); } - + setProtocolAndHost(); - + if (enableProxy) { if (started) { proxy.reopen(); } else { proxy.open(getBaseUrl().toURI()); } - } - + } + } finally { started = true; if (prevContext != null) { @@ -548,7 +548,7 @@ public class JettySolrRunner { this.protocol = protocol; this.host = c.getHost(); } - + private void retryOnPortBindFailure(int portRetryTime, int port) throws Exception, InterruptedException { TimeOut timeout = new TimeOut(portRetryTime, TimeUnit.SECONDS, TimeSource.NANO_TIME); int tryCnt = 1; @@ -567,7 +567,7 @@ public class JettySolrRunner { continue; } } - + throw e; } } @@ -628,7 +628,7 @@ public class JettySolrRunner { QueuedThreadPool qtp = (QueuedThreadPool) server.getThreadPool(); ReservedThreadExecutor rte = qtp.getBean(ReservedThreadExecutor.class); - + server.stop(); if (server.getState().equals(Server.FAILED)) { @@ -647,18 +647,18 @@ public class JettySolrRunner { Thread.sleep(50); } } - + // we tried to kill everything, now we wait for executor to stop qtp.setStopTimeout(Integer.MAX_VALUE); qtp.stop(); qtp.join(); - + if (rte != null) { // we try and wait for the reserved thread executor, but it doesn't always seem to work // so we actually set 0 reserved threads at creation - + rte.stop(); - + TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME); timeout.waitFor("Timeout waiting for reserved executor to stop.", () -> rte.isStopped()); @@ -675,12 +675,12 @@ public class JettySolrRunner { // ignore } } while (!server.isStopped()); - + } finally { if (enableProxy) { proxy.close(); } - + if (prevContext != null) { MDC.setContextMap(prevContext); } else { @@ -691,7 +691,7 @@ public class JettySolrRunner { /** * Returns the Local Port of the jetty Server. - * + * * @exception RuntimeException if there is no Connector */ private int getFirstConnectorPort() { @@ -701,22 +701,22 @@ public class JettySolrRunner { } return ((ServerConnector) conns[0]).getLocalPort(); } - - + + /** * Returns the Local Port of the jetty Server. - * + * * @exception RuntimeException if there is no Connector */ public int getLocalPort() { return getLocalPort(false); } - + /** * Returns the Local Port of the jetty Server. - * + * * @param internalPort pass true to get the true jetty port rather than the proxy port if configured - * + * * @exception RuntimeException if there is no Connector */ public int getLocalPort(boolean internalPort) { @@ -728,7 +728,7 @@ public class JettySolrRunner { } return (proxyPort != -1) ? proxyPort : jettyPort; } - + /** * Sets the port of a local socket proxy that sits infront of this server; if set * then all client traffic will flow through the proxy, giving us the ability to @@ -737,7 +737,7 @@ public class JettySolrRunner { public void setProxyPort(int proxyPort) { this.proxyPort = proxyPort; } - + /** * Returns a base URL consisting of the protocol, host, and port for a * Connector in use by the Jetty Server contained in this runner. @@ -764,7 +764,7 @@ public class JettySolrRunner { public SolrClient newClient() { return new HttpSolrClient.Builder(getBaseUrl().toString()).build(); } - + public SolrClient newClient(int connectionTimeoutMillis, int socketTimeoutMillis) { return new HttpSolrClient.Builder(getBaseUrl().toString()) .withConnectionTimeout(connectionTimeoutMillis) @@ -793,13 +793,9 @@ public class JettySolrRunner { /** * A main class that starts jetty+solr This is useful for debugging */ - public static void main(String[] args) { - try { - JettySolrRunner jetty = new JettySolrRunner(".", "/solr", 8983); - jetty.start(); - } catch (Exception ex) { - ex.printStackTrace(); - } + public static void main(String[] args) throws Exception { + JettySolrRunner jetty = new JettySolrRunner(".", "/solr", 8983); + jetty.start(); } /** @@ -829,12 +825,12 @@ public class JettySolrRunner { throw new IllegalStateException("The dispatchFilter is not set!"); } } - + static class Delay { final AtomicInteger counter; final int delayValue; final String reason; - + public Delay(String reason, int counter, int delay) { this.reason = reason; this.counter = new AtomicInteger(counter); diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java b/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java index 2df87a03c72..214fd60d9aa 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java @@ -43,6 +43,7 @@ import org.apache.solr.common.cloud.ClusterProperties; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkConfigManager; import org.apache.solr.core.CoreContainer; +import org.apache.solr.util.CLIO; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.xml.sax.SAXException; @@ -50,8 +51,8 @@ import org.xml.sax.SAXException; import static org.apache.solr.common.params.CommonParams.NAME; import static org.apache.solr.common.params.CommonParams.VALUE_LONG; -public class ZkCLI { - +public class ZkCLI implements CLIO { + private static final String MAKEPATH = "makepath"; private static final String PUT = "put"; private static final String PUT_FILE = "putfile"; @@ -84,19 +85,19 @@ public class ZkCLI { ZkCLI.stdout = stdout; } - private static PrintStream stdout = System.out; - + private static PrintStream stdout = CLIO.getOutStream(); + /** * Allows you to perform a variety of zookeeper related tasks, such as: - * + * * Bootstrap the current configs for all collections in solr.xml. - * + * * Upload a named config set from a given directory. - * + * * Link a named config set explicity to a collection. - * + * * Clear ZooKeeper info. - * + * * If you also pass a solrPort, it will be used to start an embedded zk useful * for single machine, multi node tests. */ @@ -106,7 +107,7 @@ public class ZkCLI { CommandLineParser parser = new PosixParser(); Options options = new Options(); - + options.addOption(OptionBuilder .hasArg(true) .withDescription( @@ -121,16 +122,16 @@ public class ZkCLI { Option solrHomeOption = new Option("s", SOLRHOME, true, "for " + BOOTSTRAP + ", " + RUNZK + ": solrhome location"); options.addOption(solrHomeOption); - + options.addOption("d", CONFDIR, true, "for " + UPCONFIG + ": a directory of configuration files"); options.addOption("n", CONFNAME, true, "for " + UPCONFIG + ", " + LINKCONFIG + ": name of the config set"); - + options.addOption("c", COLLECTION, true, "for " + LINKCONFIG + ": name of the collection"); - + options.addOption(EXCLUDE_REGEX_SHORT, EXCLUDE_REGEX, true, "for " + UPCONFIG + ": files matching this regular expression won't be uploaded"); @@ -140,7 +141,7 @@ public class ZkCLI { RUNZK, true, "run zk internally by passing the solr run port - only for clusters on one machine (tests, dev)"); - + options.addOption("h", HELP, false, "bring up this help page"); options.addOption(NAME, true, "name of the cluster property to set"); options.addOption(VALUE_LONG, true, "value of the cluster to set"); @@ -148,7 +149,7 @@ public class ZkCLI { try { // parse the command line arguments CommandLine line = parser.parse(options, args); - + if (line.hasOption(HELP) || !line.hasOption(ZKHOST) || !line.hasOption(CMD)) { // automatically generate the help statement @@ -171,11 +172,11 @@ public class ZkCLI { stdout.println("zkcli.sh -zkhost localhost:9983 -cmd " + UPDATEACLS + " /solr"); return; } - + // start up a tmp zk server first String zkServerAddress = line.getOptionValue(ZKHOST); String solrHome = line.getOptionValue(SOLRHOME); - + String solrPort = null; if (line.hasOption(RUNZK)) { if (!line.hasOption(SOLRHOME)) { @@ -184,7 +185,7 @@ public class ZkCLI { } solrPort = line.getOptionValue(RUNZK); } - + SolrZkServer zkServer = null; if (solrPort != null) { zkServer = new SolrZkServer("true", null, solrHome + "/zoo_data", @@ -197,7 +198,7 @@ public class ZkCLI { zkClient = new SolrZkClient(zkServerAddress, 30000, 30000, () -> { }); - + if (line.getOptionValue(CMD).equalsIgnoreCase(BOOTSTRAP)) { if (!line.hasOption(SOLRHOME)) { stdout.println("-" + SOLRHOME @@ -216,7 +217,7 @@ public class ZkCLI { // No need to close the CoreContainer, as it wasn't started // up in the first place... - + } else if (line.getOptionValue(CMD).equalsIgnoreCase(UPCONFIG)) { if (!line.hasOption(CONFDIR) || !line.hasOption(CONFNAME)) { stdout.println("-" + CONFDIR + " and -" + CONFNAME @@ -226,7 +227,7 @@ public class ZkCLI { String confDir = line.getOptionValue(CONFDIR); String confName = line.getOptionValue(CONFNAME); final String excludeExpr = line.getOptionValue(EXCLUDE_REGEX, EXCLUDE_REGEX_DEFAULT); - + if(!ZkController.checkChrootPath(zkServerAddress, true)) { stdout.println("A chroot was specified in zkHost but the znode doesn't exist. "); System.exit(1); @@ -252,7 +253,7 @@ public class ZkCLI { } String collection = line.getOptionValue(COLLECTION); String confName = line.getOptionValue(CONFNAME); - + ZkController.linkConfSet(zkClient, collection, confName); } else if (line.getOptionValue(CMD).equalsIgnoreCase(LIST)) { zkClient.printLayoutToStream(stdout); @@ -368,6 +369,6 @@ public class ZkCLI { } catch (ParseException exp) { stdout.println("Unexpected exception:" + exp.getMessage()); } - + } } 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 7f28643b023..61b57f3f7fa 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -298,8 +298,8 @@ public class ZkController implements Closeable { this.genericCoreNodeNames = cloudConfig.getGenericCoreNodeNames(); // be forgiving and strip this off leading/trailing slashes - // this allows us to support users specifying hostContext="/" in - // solr.xml to indicate the root context, instead of hostContext="" + // this allows us to support users specifying hostContext="/" in + // solr.xml to indicate the root context, instead of hostContext="" // which means the default of "solr" String localHostContext = trimLeadingAndTrailingSlashes(cloudConfig.getSolrHostContext()); @@ -350,7 +350,7 @@ public class ZkController implements Closeable { // seems we dont need to do this again... // Overseer.createClientNodes(zkClient, getNodeName()); - + // start the overseer first as following code may need it's processing if (!zkRunOnly) { ElectionContext context = new OverseerElectionContext(zkClient, @@ -458,7 +458,7 @@ public class ZkController implements Closeable { }); init(registerOnReconnect); - + this.overseerJobQueue = overseer.getStateUpdateQueue(); this.overseerCollectionQueue = overseer.getCollectionQueue(zkClient); this.overseerConfigSetQueue = overseer.getConfigSetQueue(zkClient); @@ -482,7 +482,7 @@ public class ZkController implements Closeable { if (descriptors != null) { // before registering as live, make sure everyone is in a // down state - publishNodeAsDown(getNodeName()); + publishNodeAsDown(getNodeName()); for (CoreDescriptor descriptor : descriptors) { // if it looks like we are going to be the leader, we don't // want to wait for the following stuff @@ -524,9 +524,9 @@ public class ZkController implements Closeable { } } } - + private void closeOutstandingElections(final CurrentCoreDescriptorProvider registerOnReconnect) { - + List descriptors = registerOnReconnect.getCurrentDescriptors(); if (descriptors != null) { for (CoreDescriptor descriptor : descriptors) { @@ -534,20 +534,20 @@ public class ZkController implements Closeable { } } } - + private ContextKey closeExistingElectionContext(CoreDescriptor cd) { // look for old context - if we find it, cancel it String collection = cd.getCloudDescriptor().getCollectionName(); final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName(); - + ContextKey contextKey = new ContextKey(collection, coreNodeName); ElectionContext prevContext = electionContexts.get(contextKey); - + if (prevContext != null) { prevContext.close(); electionContexts.remove(contextKey); } - + return contextKey; } @@ -1002,7 +1002,7 @@ public class ZkController implements Closeable { InterruptedException { publishAndWaitForDownStates(WAIT_DOWN_STATES_TIMEOUT_SECONDS); } - + public void publishAndWaitForDownStates(int timeoutSeconds) throws KeeperException, InterruptedException { @@ -1104,7 +1104,7 @@ public class ZkController implements Closeable { List ops = new ArrayList<>(2); ops.add(Op.delete(nodePath, -1)); ops.add(Op.delete(nodeAddedPath, -1)); - + try { zkClient.multi(ops, true); } catch (NoNodeException e) { @@ -1194,25 +1194,25 @@ public class ZkController implements Closeable { } catch (KeeperException | IOException e) { throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e); } - + // in this case, we want to wait for the leader as long as the leader might // wait for a vote, at least - but also long enough that a large cluster has // time to get its act together String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000); - + String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName); log.debug("We are " + ourUrl + " and leader is " + leaderUrl); boolean isLeader = leaderUrl.equals(ourUrl); assert !(isLeader && replica.getType() == Type.PULL) : "Pull replica became leader!"; try (SolrCore core = cc.getCore(desc.getName())) { - + // recover from local transaction log and wait for it to complete before // going active // TODO: should this be moved to another thread? To recoveryStrat? // TODO: should this actually be done earlier, before (or as part of) // leader election perhaps? - + if (core == null) { throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "SolrCore is no longer available to register"); } @@ -1260,7 +1260,7 @@ public class ZkController implements Closeable { unregister(coreName, desc, false); throw e; } - + // make sure we have an update cluster state right away zkStateReader.forceUpdateCollection(collection); // the watcher is added to a set so multiple calls of this method will left only one watcher @@ -1350,7 +1350,7 @@ public class ZkController implements Closeable { .getCoreUrl(); } - } catch (AlreadyClosedException e) { + } catch (AlreadyClosedException e) { throw e; } catch (Exception e) { log.error("Error getting leader from zk", e); @@ -1502,7 +1502,7 @@ public class ZkController implements Closeable { } try { String collection = cd.getCloudDescriptor().getCollectionName(); - + log.debug("publishing state={}", state.toString()); // System.out.println(Thread.currentThread().getStackTrace()[3]); Integer numShards = cd.getCloudDescriptor().getNumShards(); @@ -1510,11 +1510,11 @@ public class ZkController implements Closeable { log.debug("numShards not found on descriptor - reading it from system property"); numShards = Integer.getInteger(ZkStateReader.NUM_SHARDS_PROP); } - + assert collection != null && collection.length() > 0; - + String shardId = cd.getCloudDescriptor().getShardId(); - + String coreNodeName = cd.getCloudDescriptor().getCoreNodeName(); Map props = new HashMap<>(); @@ -1566,7 +1566,7 @@ public class ZkController implements Closeable { } ZkNodeProps m = new ZkNodeProps(props); - + if (updateLastState) { cd.getCloudDescriptor().setLastPublished(state); } @@ -1638,11 +1638,6 @@ public class ZkController implements Closeable { overseerJobQueue.offer(Utils.toJSON(m)); } - // convenience for testing - void printLayoutToStdOut() throws KeeperException, InterruptedException { - zkClient.printLayoutToStdOut(); - } - public ZkStateReader getZkStateReader() { return zkStateReader; } @@ -1842,7 +1837,7 @@ public class ZkController implements Closeable { CoreDescriptor descriptor, final String coreZkNodeName) { // try not to wait too long here - if we are waiting too long, we should probably // move along and join the election - + CloudDescriptor cloudDesc = descriptor.getCloudDescriptor(); String collection = cloudDesc.getCollectionName(); String shard = cloudDesc.getShardId(); @@ -2044,7 +2039,7 @@ public class ZkController implements Closeable { * has been reserved for the operation, meaning that no other thread/operation can claim * it. If for whatever reason, the operation is not scheduled, the asuncId needs to be * cleared using {@link #clearAsyncId(String)}. - * If this method returns false, no reservation has been made, and this asyncId can't + * If this method returns false, no reservation has been made, and this asyncId can't * be used, since it's being used by another operation (currently or in the past) * @param asyncId A string representing the asyncId of an operation. Can't be null. * @return True if the reservation succeeds. @@ -2059,7 +2054,7 @@ public class ZkController implements Closeable { throw new RuntimeException(e); } } - + /** * Clears an asyncId previously claimed by calling {@link #claimAsyncId(String)} * @param asyncId A string representing the asyncId of an operation. Can't be null. @@ -2161,7 +2156,7 @@ public class ZkController implements Closeable { public void rejoinShardLeaderElection(SolrParams params) { try { - + String collectionName = params.get(COLLECTION_PROP); String shardId = params.get(SHARD_ID_PROP); String coreNodeName = params.get(CORE_NODE_NAME_PROP); @@ -2171,24 +2166,24 @@ public class ZkController implements Closeable { try (SolrCore core = cc.getCore(coreName)) { MDCLoggingContext.setCore(core); - + log.info("Rejoin the shard leader election."); - + ContextKey contextKey = new ContextKey(collectionName, coreNodeName); - + ElectionContext prevContext = electionContexts.get(contextKey); if (prevContext != null) prevContext.cancelElection(); - + ZkNodeProps zkProps = new ZkNodeProps(BASE_URL_PROP, baseUrl, CORE_NAME_PROP, coreName, NODE_NAME_PROP, getNodeName(), CORE_NODE_NAME_PROP, coreNodeName); - + LeaderElector elect = ((ShardLeaderElectionContextBase) prevContext).getLeaderElector(); ShardLeaderElectionContext context = new ShardLeaderElectionContext(elect, shardId, collectionName, coreNodeName, zkProps, this, getCoreContainer()); - + context.leaderSeqPath = context.electionPath + LeaderElector.ELECTION_NODE + "/" + electionNode; elect.setup(context); electionContexts.put(contextKey, context); - + elect.retryElection(context, params.getBool(REJOIN_AT_HEAD_PROP, false)); } } catch (Exception e) { @@ -2393,7 +2388,7 @@ public class ZkController implements Closeable { public void preClose(SolrCore core) { unregisterConfListener(confDir, listener); } - + @Override public void postClose(SolrCore core) { } @@ -2584,11 +2579,11 @@ public class ZkController implements Closeable { } return false; } - - + + /** * Best effort to set DOWN state for all replicas on node. - * + * * @param nodeName to operate on */ public void publishNodeAsDown(String nodeName) { @@ -2604,7 +2599,7 @@ public class ZkController implements Closeable { log.debug("Publish node as down was interrupted."); } catch (KeeperException e) { log.warn("Could not publish node as down: " + e.getMessage()); - } + } } /** diff --git a/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java b/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java index 34729995c63..cb13efdbf6a 100644 --- a/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java +++ b/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java @@ -262,8 +262,4 @@ public class ConfigOverlay implements MapSerializable { public static final String ZNODEVER = "znodeVersion"; public static final String NAME = "overlay"; - public static void main(String[] args) { - System.out.println(Utils.toJSONString(editable_prop_map)); - } - } diff --git a/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java b/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java index 062f434b65c..88557f690d4 100644 --- a/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java +++ b/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotsTool.java @@ -58,6 +58,7 @@ import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.params.CollectionAdminParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.core.snapshots.CollectionSnapshotMetaData.CoreSnapshotMetaData; +import org.apache.solr.util.CLIO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,7 +67,7 @@ import com.google.common.base.Preconditions; /** * This class provides utility functions required for Solr snapshots functionality. */ -public class SolrSnapshotsTool implements Closeable { +public class SolrSnapshotsTool implements Closeable, CLIO { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final DateFormat dateFormat = new SimpleDateFormat("EEE, d MMM yyyy HH:mm:ss z", Locale.getDefault()); @@ -107,11 +108,11 @@ public class SolrSnapshotsTool implements Closeable { try { resp = createSnap.process(solrClient); Preconditions.checkState(resp.getStatus() == 0, "The CREATESNAPSHOT request failed. The status code is " + resp.getStatus()); - System.out.println("Successfully created snapshot with name " + snapshotName + " for collection " + collectionName); + CLIO.out("Successfully created snapshot with name " + snapshotName + " for collection " + collectionName); } catch (Exception e) { log.error("Failed to create a snapshot with name " + snapshotName + " for collection " + collectionName, e); - System.out.println("Failed to create a snapshot with name " + snapshotName + " for collection " + collectionName + CLIO.out("Failed to create a snapshot with name " + snapshotName + " for collection " + collectionName +" due to following error : "+e.getLocalizedMessage()); } } @@ -122,11 +123,11 @@ public class SolrSnapshotsTool implements Closeable { try { resp = deleteSnap.process(solrClient); Preconditions.checkState(resp.getStatus() == 0, "The DELETESNAPSHOT request failed. The status code is " + resp.getStatus()); - System.out.println("Successfully deleted snapshot with name " + snapshotName + " for collection " + collectionName); + CLIO.out("Successfully deleted snapshot with name " + snapshotName + " for collection " + collectionName); } catch (Exception e) { log.error("Failed to delete a snapshot with name " + snapshotName + " for collection " + collectionName, e); - System.out.println("Failed to delete a snapshot with name " + snapshotName + " for collection " + collectionName + CLIO.out("Failed to delete a snapshot with name " + snapshotName + " for collection " + collectionName +" due to following error : "+e.getLocalizedMessage()); } } @@ -141,12 +142,12 @@ public class SolrSnapshotsTool implements Closeable { NamedList apiResult = (NamedList) resp.getResponse().get(SolrSnapshotManager.SNAPSHOTS_INFO); for (int i = 0; i < apiResult.size(); i++) { - System.out.println(apiResult.getName(i)); + CLIO.out(apiResult.getName(i)); } } catch (Exception e) { log.error("Failed to list snapshots for collection " + collectionName, e); - System.out.println("Failed to list snapshots for collection " + collectionName + CLIO.out("Failed to list snapshots for collection " + collectionName +" due to following error : "+e.getLocalizedMessage()); } } @@ -156,11 +157,11 @@ public class SolrSnapshotsTool implements Closeable { Collection snaps = listCollectionSnapshots(collectionName); for (CollectionSnapshotMetaData m : snaps) { if (snapshotName.equals(m.getName())) { - System.out.println("Name: " + m.getName()); - System.out.println("Status: " + m.getStatus()); - System.out.println("Time of creation: " + dateFormat.format(m.getCreationDate())); - System.out.println("Total number of cores with snapshot: " + m.getReplicaSnapshots().size()); - System.out.println("-----------------------------------"); + CLIO.out("Name: " + m.getName()); + CLIO.out("Status: " + m.getStatus()); + CLIO.out("Time of creation: " + dateFormat.format(m.getCreationDate())); + CLIO.out("Total number of cores with snapshot: " + m.getReplicaSnapshots().size()); + CLIO.out("-----------------------------------"); for (CoreSnapshotMetaData n : m.getReplicaSnapshots()) { StringBuilder builder = new StringBuilder(); builder.append("Core [name="); @@ -172,13 +173,13 @@ public class SolrSnapshotsTool implements Closeable { builder.append(", indexDirPath="); builder.append(n.getIndexDirPath()); builder.append("]\n"); - System.out.println(builder.toString()); + CLIO.out(builder.toString()); } } } } catch (Exception e) { log.error("Failed to fetch snapshot details", e); - System.out.println("Failed to fetch snapshot details due to following error : " + e.getLocalizedMessage()); + CLIO.out("Failed to fetch snapshot details due to following error : " + e.getLocalizedMessage()); } } @@ -272,21 +273,21 @@ public class SolrSnapshotsTool implements Closeable { public void prepareForExport(String collectionName, String snapshotName, String localFsPath, Optional pathPrefix, String destPath) { try { buildCopyListings(collectionName, snapshotName, localFsPath, pathPrefix); - System.out.println("Successfully prepared copylisting for the snapshot export."); + CLIO.out("Successfully prepared copylisting for the snapshot export."); } catch (Exception e) { log.error("Failed to prepare a copylisting for snapshot with name " + snapshotName + " for collection " + collectionName, e); - System.out.println("Failed to prepare a copylisting for snapshot with name " + snapshotName + " for collection " + CLIO.out("Failed to prepare a copylisting for snapshot with name " + snapshotName + " for collection " + collectionName + " due to following error : " + e.getLocalizedMessage()); System.exit(1); } try { backupCollectionMetaData(collectionName, snapshotName, destPath); - System.out.println("Successfully backed up collection meta-data"); + CLIO.out("Successfully backed up collection meta-data"); } catch (Exception e) { log.error("Failed to backup collection meta-data for collection " + collectionName, e); - System.out.println("Failed to backup collection meta-data for collection " + collectionName + CLIO.out("Failed to backup collection meta-data for collection " + collectionName + " due to following error : " + e.getLocalizedMessage()); System.exit(1); } @@ -306,7 +307,7 @@ public class SolrSnapshotsTool implements Closeable { backup.processAsync(asyncReqId.orElse(null), solrClient); } catch (Exception e) { log.error("Failed to backup collection meta-data for collection " + collectionName, e); - System.out.println("Failed to backup collection meta-data for collection " + collectionName + CLIO.out("Failed to backup collection meta-data for collection " + collectionName + " due to following error : " + e.getLocalizedMessage()); System.exit(1); } @@ -342,7 +343,7 @@ public class SolrSnapshotsTool implements Closeable { try { cmd = parser.parse(options, args); } catch (ParseException e) { - System.out.println(e.getLocalizedMessage()); + CLIO.out(e.getLocalizedMessage()); printHelp(options); System.exit(1); } @@ -380,7 +381,7 @@ public class SolrSnapshotsTool implements Closeable { try { new URI(pathPrefix.get()); } catch (URISyntaxException e) { - System.out.println( + CLIO.out( "The specified File system path prefix " + pathPrefix.get() + " is invalid. The error is " + e.getLocalizedMessage()); System.exit(1); @@ -401,14 +402,14 @@ public class SolrSnapshotsTool implements Closeable { } else if (cmd.hasOption(HELP)) { printHelp(options); } else { - System.out.println("Unknown command specified."); + CLIO.out("Unknown command specified."); printHelp(options); } } private static String requiredArg(Options options, CommandLine cmd, String optVal) { if (!cmd.hasOption(optVal)) { - System.out.println("Please specify the value for option " + optVal); + CLIO.out("Please specify the value for option " + optVal); printHelp(options); System.exit(1); } diff --git a/solr/core/src/java/org/apache/solr/parser/QueryParserTokenManager.java b/solr/core/src/java/org/apache/solr/parser/QueryParserTokenManager.java index 6251ff473e4..16676431a1f 100644 --- a/solr/core/src/java/org/apache/solr/parser/QueryParserTokenManager.java +++ b/solr/core/src/java/org/apache/solr/parser/QueryParserTokenManager.java @@ -6,10 +6,6 @@ public class QueryParserTokenManager implements QueryParserConstants { int commentNestingDepth ; - /** Debug output. */ - public java.io.PrintStream debugStream = System.out; - /** Set debug output. */ - public void setDebugStream(java.io.PrintStream ds) { debugStream = ds; } private final int jjStopStringLiteralDfa_3(int pos, long active0) { switch (pos) @@ -1332,9 +1328,9 @@ private int jjMoveNfa_1(int startState, int curPos) } } static final int[] jjnextStates = { - 32, 34, 35, 31, 36, 17, 18, 20, 56, 59, 25, 60, 57, 59, 25, 60, - 22, 23, 38, 39, 46, 38, 39, 40, 46, 38, 39, 41, 49, 54, 46, 42, - 43, 45, 50, 51, 53, 38, 39, 54, 46, 58, 61, 29, 2, 4, 5, + 32, 34, 35, 31, 36, 17, 18, 20, 56, 59, 25, 60, 57, 59, 25, 60, + 22, 23, 38, 39, 46, 38, 39, 40, 46, 38, 39, 41, 49, 54, 46, 42, + 43, 45, 50, 51, 53, 38, 39, 54, 46, 58, 61, 29, 2, 4, 5, }; private static final boolean jjCanMove_0(int hiByte, int i1, int i2, long l1, long l2) { @@ -1375,9 +1371,9 @@ private static final boolean jjCanMove_2(int hiByte, int i1, int i2, long l1, lo /** Token literal values. */ public static final String[] jjstrLiteralImages = { -"", null, null, null, null, null, null, null, null, null, null, null, null, -null, null, null, "\53", "\55", null, "\50", "\51", "\72", "\52", "\136", null, null, -null, null, null, null, "\133", "\173", null, "\146\151\154\164\145\162\50", null, +"", null, null, null, null, null, null, null, null, null, null, null, null, +null, null, null, "\53", "\55", null, "\50", "\51", "\72", "\52", "\136", null, null, +null, null, null, null, "\133", "\173", null, "\146\151\154\164\145\162\50", null, "\124\117", "\135", "\175", null, null, }; /** Lexer state names. */ @@ -1390,14 +1386,14 @@ public static final String[] lexStateNames = { /** Lex State array. */ public static final int[] jjnewLexState = { - -1, -1, -1, -1, -1, -1, -1, -1, -1, 2, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, -1, - -1, -1, -1, -1, -1, 1, 1, -1, -1, 3, -1, 3, 3, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 2, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, -1, + -1, -1, -1, -1, -1, 1, 1, -1, -1, 3, -1, 3, 3, -1, -1, }; static final long[] jjtoToken = { - 0xffffffe001L, + 0xffffffe001L, }; static final long[] jjtoSkip = { - 0x1f00L, + 0x1f00L, }; protected CharStream input_stream; private final int[] jjrounds = new int[63]; @@ -1482,7 +1478,7 @@ int jjmatchedPos; int jjmatchedKind; /** Get the next Token. */ -public Token getNextToken() +public Token getNextToken() { Token matchedToken; int curPos = 0; diff --git a/solr/core/src/java/org/apache/solr/response/XMLWriter.java b/solr/core/src/java/org/apache/solr/response/XMLWriter.java index 777b8b1e171..d8e35270440 100644 --- a/solr/core/src/java/org/apache/solr/response/XMLWriter.java +++ b/solr/core/src/java/org/apache/solr/response/XMLWriter.java @@ -18,6 +18,7 @@ package org.apache.solr.response; import java.io.IOException; import java.io.Writer; +import java.lang.invoke.MethodHandles; import java.util.Arrays; import java.util.Iterator; import java.util.Map; @@ -31,6 +32,8 @@ import org.apache.solr.common.util.XML; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.search.ReturnFields; import org.apache.solr.search.SolrReturnFields; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.solr.common.params.CommonParams.NAME; @@ -39,6 +42,7 @@ import static org.apache.solr.common.params.CommonParams.NAME; * @lucene.internal */ public class XMLWriter extends TextResponseWriter { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); public static float CURRENT_VERSION=2.2f; @@ -53,7 +57,7 @@ public class XMLWriter extends TextResponseWriter { +" xsi:noNamespaceSchemaLocation=\"http://pi.cnet.com/cnet-search/response.xsd\">\n" ).toCharArray(); ***/ - + private static final char[] XML_START2_NOSCHEMA=("\n").toCharArray(); final int version; @@ -162,7 +166,7 @@ public class XMLWriter extends TextResponseWriter { @Override - public void writeStartDocumentList(String name, + public void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore) throws IOException { if (doIndent) indent(); @@ -175,7 +179,7 @@ public class XMLWriter extends TextResponseWriter { writeAttr("maxScore",Float.toString(maxScore)); } writer.write(">"); - + incLevel(); } @@ -183,7 +187,7 @@ public class XMLWriter extends TextResponseWriter { /** * The SolrDocument should already have multivalued fields implemented as * Collections -- this will not rewrite to <arr> - */ + */ @Override public void writeSolrDocument(String name, SolrDocument doc, ReturnFields returnFields, int idx ) throws IOException { startTag("doc", name, false); @@ -196,7 +200,7 @@ public class XMLWriter extends TextResponseWriter { Object val = doc.getFieldValue(fname); if( "_explain_".equals( fname ) ) { - System.out.println( val ); + log.debug(String.valueOf(val)); } writeVal(fname, val); } @@ -206,11 +210,11 @@ public class XMLWriter extends TextResponseWriter { writeSolrDocument(null, childDoc, new SolrReturnFields(), idx); } } - + decLevel(); writer.write(""); } - + @Override public void writeEndDocumentList() throws IOException { diff --git a/solr/core/src/java/org/apache/solr/response/transform/ExplainAugmenterFactory.java b/solr/core/src/java/org/apache/solr/response/transform/ExplainAugmenterFactory.java index c35ebd552ef..4222da4a09b 100644 --- a/solr/core/src/java/org/apache/solr/response/transform/ExplainAugmenterFactory.java +++ b/solr/core/src/java/org/apache/solr/response/transform/ExplainAugmenterFactory.java @@ -107,22 +107,17 @@ public class ExplainAugmenterFactory extends TransformerFactory } @Override - public void transform(SolrDocument doc, int docid) { + public void transform(SolrDocument doc, int docid) throws IOException { if( context != null && context.getQuery() != null ) { - try { - Explanation exp = context.getSearcher().explain(context.getQuery(), docid); - if( style == Style.nl ) { - doc.setField( name, SolrPluginUtils.explanationToNamedList(exp) ); - } - else if( style == Style.html ) { - doc.setField( name, toHtml(exp)); - } - else { - doc.setField( name, exp.toString() ); - } + Explanation exp = context.getSearcher().explain(context.getQuery(), docid); + if( style == Style.nl ) { + doc.setField( name, SolrPluginUtils.explanationToNamedList(exp) ); } - catch (IOException e) { - e.printStackTrace(); + else if( style == Style.html ) { + doc.setField( name, toHtml(exp)); + } + else { + doc.setField( name, exp.toString() ); } } } diff --git a/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java b/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java index 7669db8634a..f7a79dc82d1 100644 --- a/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java +++ b/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java @@ -172,8 +172,7 @@ public class CloudMLTQParser extends QParser { return realMLTQuery.build(); } catch (IOException e) { - e.printStackTrace(); - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Bad Request"); + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Bad Request", e); } } diff --git a/solr/core/src/java/org/apache/solr/util/CLIO.java b/solr/core/src/java/org/apache/solr/util/CLIO.java new file mode 100644 index 00000000000..80c3675d033 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/util/CLIO.java @@ -0,0 +1,40 @@ +/* + * 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.io.PrintStream; + +import org.apache.solr.common.util.SuppressForbidden; + +@SuppressForbidden( reason = "For use in command line tools only") +public interface CLIO { + static void out(String s) { + System.out.println(s); + } + + static void err(String s) { + System.err.println(s); + } + + static PrintStream getOutStream() { + return System.out; + } + + static PrintStream getErrStream() { + return System.err; + } +} diff --git a/solr/core/src/java/org/apache/solr/util/CryptoKeys.java b/solr/core/src/java/org/apache/solr/util/CryptoKeys.java index faf67fda306..0835b59d987 100644 --- a/solr/core/src/java/org/apache/solr/util/CryptoKeys.java +++ b/solr/core/src/java/org/apache/solr/util/CryptoKeys.java @@ -50,7 +50,7 @@ import org.slf4j.LoggerFactory; /**A utility class to verify signatures * */ -public final class CryptoKeys { +public final class CryptoKeys implements CLIO { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private final Map keys; private Exception exception; @@ -342,14 +342,14 @@ public final class CryptoKeys { public static void main(String[] args) throws Exception { RSAKeyPair keyPair = new RSAKeyPair(); - System.out.println(keyPair.getPublicKeyStr()); + CLIO.out(keyPair.getPublicKeyStr()); PublicKey pk = deserializeX509PublicKey(keyPair.getPublicKeyStr()); byte[] payload = "Hello World!".getBytes(StandardCharsets.UTF_8); byte[] encrypted = keyPair.encrypt(ByteBuffer.wrap(payload)); String cipherBase64 = Base64.byteArrayToBase64(encrypted); - System.out.println("encrypted: "+ cipherBase64); - System.out.println("signed: "+ Base64.byteArrayToBase64(keyPair.signSha256(payload))); - System.out.println("decrypted "+ new String(decryptRSA(encrypted , pk), StandardCharsets.UTF_8)); + CLIO.out("encrypted: "+ cipherBase64); + CLIO.out("signed: "+ Base64.byteArrayToBase64(keyPair.signSha256(payload))); + CLIO.out("decrypted "+ new String(decryptRSA(encrypted , pk), StandardCharsets.UTF_8)); } } diff --git a/solr/core/src/java/org/apache/solr/util/RecordingJSONParser.java b/solr/core/src/java/org/apache/solr/util/RecordingJSONParser.java index a85610b7cb2..932ae5fbffd 100644 --- a/solr/core/src/java/org/apache/solr/util/RecordingJSONParser.java +++ b/solr/core/src/java/org/apache/solr/util/RecordingJSONParser.java @@ -18,10 +18,15 @@ package org.apache.solr.util; import java.io.IOException; import java.io.Reader; +import java.lang.invoke.MethodHandles; import org.noggit.JSONParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RecordingJSONParser extends JSONParser { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + static ThreadLocal buf = new ThreadLocal<>(); private final char[] bufCopy; //global position is the global position at the beginning of my buffer @@ -68,7 +73,7 @@ public class RecordingJSONParser extends JSONParser { private void captureMissing() { long currPosition = getPosition() - globalPosition; if(currPosition < 0){ - System.out.println("ERROR"); + log.error("currPosition less than zero in captureMissing()?"); } if (currPosition > lastMarkedPosition) { diff --git a/solr/core/src/java/org/apache/solr/util/SimplePostTool.java b/solr/core/src/java/org/apache/solr/util/SimplePostTool.java index a1d71a5191b..fa429b84eb7 100644 --- a/solr/core/src/java/org/apache/solr/util/SimplePostTool.java +++ b/solr/core/src/java/org/apache/solr/util/SimplePostTool.java @@ -70,9 +70,9 @@ import static java.nio.charset.StandardCharsets.US_ASCII; import static java.nio.charset.StandardCharsets.UTF_8; /** - * A simple utility class for posting raw updates to a Solr server, + * A simple utility class for posting raw updates to a Solr server, * has a main method so it can be run on the command line. - * View this not as a best-practice code example, but as a standalone + * View this not as a best-practice code example, but as a standalone * example built with an explicit purpose of not having external * jar dependencies. */ @@ -119,7 +119,7 @@ public class SimplePostTool { // Backlog for crawling List> backlog = new ArrayList<>(); Set visited = new HashSet<>(); - + static final Set DATA_MODES = new HashSet<>(); static final String USAGE_STRING_SHORT = "Usage: java [SystemProperties] -jar post.jar [-h|-] [ [...]]"; @@ -133,7 +133,7 @@ public class SimplePostTool { DATA_MODES.add(DATA_MODE_ARGS); DATA_MODES.add(DATA_MODE_STDIN); DATA_MODES.add(DATA_MODE_WEB); - + mimeMap = new HashMap<>(); mimeMap.put("xml", "application/xml"); mimeMap.put("csv", "text/csv"); @@ -158,7 +158,7 @@ public class SimplePostTool { mimeMap.put("txt", "text/plain"); mimeMap.put("log", "text/plain"); } - + /** * See usage() for valid command line usage * @param args the params on the command line @@ -191,12 +191,12 @@ public class SimplePostTool { usageShort(); return; } - + if (commit) commit(); if (optimize) optimize(); displayTiming((long) timer.getTime()); } - + /** * Pretty prints the number of milliseconds taken to post the content to Solr * @param millis the time in milliseconds @@ -204,7 +204,7 @@ public class SimplePostTool { private void displayTiming(long millis) { SimpleDateFormat df = new SimpleDateFormat("H:mm:ss.SSS", Locale.getDefault()); df.setTimeZone(TimeZone.getTimeZone("UTC")); - System.out.println("Time spent: "+df.format(new Date(millis))); + CLIO.out("Time spent: "+df.format(new Date(millis))); } /** @@ -220,19 +220,19 @@ public class SimplePostTool { if (! DATA_MODES.contains(mode)) { fatal("System Property 'data' is not valid for this tool: " + mode); } - + String params = System.getProperty("params", ""); String host = System.getProperty("host", DEFAULT_POST_HOST); String port = System.getProperty("port", DEFAULT_POST_PORT); String core = System.getProperty("c"); - + urlStr = System.getProperty("url"); - + if (urlStr == null && core == null) { fatal("Specifying either url or core/collection is mandatory.\n" + USAGE_STRING_SHORT); } - + if(urlStr == null) { urlStr = String.format(Locale.ROOT, "http://%s:%s/solr/%s/update", host, port, core); } @@ -246,7 +246,7 @@ public class SimplePostTool { } if (user != null) info("Basic Authentication enabled, user=" + user); - + boolean auto = isOn(System.getProperty("auto", DEFAULT_AUTO)); String type = System.getProperty("type"); String format = System.getProperty("format"); @@ -264,11 +264,11 @@ public class SimplePostTool { try { delay = Integer.parseInt(System.getProperty("delay", ""+delay)); } catch(Exception e) { } - OutputStream out = isOn(System.getProperty("out", DEFAULT_OUT)) ? System.out : null; + OutputStream out = isOn(System.getProperty("out", DEFAULT_OUT)) ? CLIO.getOutStream() : null; String fileTypes = System.getProperty("filetypes", DEFAULT_FILE_TYPES); boolean commit = isOn(System.getProperty("commit",DEFAULT_COMMIT)); boolean optimize = isOn(System.getProperty("optimize",DEFAULT_OPTIMIZE)); - + return new SimplePostTool(mode, url, auto, type, format, recursive, delay, fileTypes, out, commit, optimize, args); } catch (MalformedURLException e) { fatal("System Property 'url' is not a valid URL: " + urlStr); @@ -292,7 +292,7 @@ public class SimplePostTool { * @param args a String[] of arguments, varies between modes */ public SimplePostTool(String mode, URL url, boolean auto, String type, String format, - int recursive, int delay, String fileTypes, OutputStream out, + int recursive, int delay, String fileTypes, OutputStream out, boolean commit, boolean optimize, String[] args) { this.mode = mode; this.solrUrl = url; @@ -311,19 +311,19 @@ public class SimplePostTool { } public SimplePostTool() {} - + // // Do some action depending on which mode we have // private void doFilesMode() { currentDepth = 0; - // Skip posting files if special param "-" given + // Skip posting files if special param "-" given if (!args[0].equals("-")) { info("Posting files to [base] url " + solrUrl + (!auto?" using content-type "+(type==null?DEFAULT_CONTENT_TYPE:type):"")+"..."); if(auto) info("Entering auto mode. File endings considered are "+fileTypes); if(recursive > 0) - info("Entering recursive mode, max depth="+recursive+", delay="+delay+"s"); + info("Entering recursive mode, max depth="+recursive+", delay="+delay+"s"); int numFilesPosted = postFiles(args, 0, out, type); info(numFilesPosted + " files indexed."); } @@ -344,12 +344,12 @@ public class SimplePostTool { fatal("Specifying content-type with \"-Ddata=web\" is not supported"); } if (args[0].equals("-")) { - // Skip posting url if special param "-" given + // Skip posting url if special param "-" given return 0; } // Set Extracting handler as default solrUrl = appendUrlPath(solrUrl, "/extract"); - + info("Posting web pages to Solr url "+solrUrl); auto=true; info("Entering auto mode. Indexing pages with content-types corresponding to file endings "+fileTypes); @@ -372,7 +372,7 @@ public class SimplePostTool { private void doStdinMode() { info("POSTing stdin to " + solrUrl + "..."); - postData(System.in, null, out, type, solrUrl); + postData(System.in, null, out, type, solrUrl); } private void reset() { @@ -385,12 +385,12 @@ public class SimplePostTool { // USAGE // private static void usageShort() { - System.out.println(USAGE_STRING_SHORT+"\n"+ + CLIO.out(USAGE_STRING_SHORT+"\n"+ " Please invoke with -h option for extended usage help."); } private static void usage() { - System.out.println + CLIO.out (USAGE_STRING_SHORT+"\n\n" + "Supported System Properties and their defaults:\n"+ " -Dc=\n"+ @@ -458,14 +458,14 @@ public class SimplePostTool { File[] files = parent.listFiles(ff); if(files == null || files.length == 0) { warn("No files or directories matching "+srcFile); - continue; + continue; } filesPosted += postFiles(parent.listFiles(ff), out, type); } } return filesPosted; } - + /** Post all filenames provided in args * @param files array of Files * @param startIndexInArgs offset to start @@ -489,14 +489,14 @@ public class SimplePostTool { File[] fileList = parent.listFiles(ff); if(fileList == null || fileList.length == 0) { warn("No files or directories matching "+srcFile); - continue; + continue; } filesPosted += postFiles(fileList, out, type); } } return filesPosted; } - + /** * Posts a whole directory * @return number of files posted total @@ -603,7 +603,7 @@ public class SimplePostTool { PageFetcherResult result = pageFetcher.readPageFromUrl(u); if(result.httpStatus == 200) { u = (result.redirectUrl != null) ? result.redirectUrl : u; - URL postUrl = new URL(appendParam(solrUrl.toString(), + URL postUrl = new URL(appendParam(solrUrl.toString(), "literal.id="+URLEncoder.encode(u.toString(),"UTF-8") + "&literal.url="+URLEncoder.encode(u.toString(),"UTF-8"))); boolean success = postData(new ByteArrayInputStream(result.content.array(), result.content.arrayOffset(),result.content.limit() ), null, out, result.contentType, postUrl); @@ -632,7 +632,7 @@ public class SimplePostTool { backlog.add(subStack); numPages += webCrawl(level+1, out); } - return numPages; + return numPages; } public static class BAOS extends ByteArrayOutputStream { public ByteBuffer getByteBuffer() { @@ -726,22 +726,22 @@ public class SimplePostTool { protected static boolean isOn(String property) { return("true,on,yes,1".indexOf(property) > -1); } - + static void warn(String msg) { - System.err.println("SimplePostTool: WARNING: " + msg); + CLIO.err("SimplePostTool: WARNING: " + msg); } static void info(String msg) { - System.out.println(msg); + CLIO.out(msg); } static void fatal(String msg) { - System.err.println("SimplePostTool: FATAL: " + msg); + CLIO.err("SimplePostTool: FATAL: " + msg); System.exit(2); } /** - * Does a simple commit operation + * Does a simple commit operation */ public void commit() { info("COMMITting Solr index changes to " + solrUrl + "..."); @@ -749,7 +749,7 @@ public class SimplePostTool { } /** - * Does a simple optimize operation + * Does a simple optimize operation */ public void optimize() { info("Performing an OPTIMIZE to " + solrUrl + "..."); @@ -757,7 +757,7 @@ public class SimplePostTool { } /** - * Appends a URL query parameter to a URL + * Appends a URL query parameter to a URL * @param url the original URL * @param param the parameter(s) to append, separated by "&" * @return the string version of the resulting URL @@ -778,7 +778,7 @@ public class SimplePostTool { /** * Opens the file and posts its contents to the solrUrl, - * writes to response to output. + * writes to response to output. */ public void postFile(File file, OutputStream output, String type) { InputStream is = null; @@ -814,7 +814,6 @@ public class SimplePostTool { is = new FileInputStream(file); postData(is, file.length(), output, type, url); } catch (IOException e) { - e.printStackTrace(); warn("Can't open/read file: " + file); } finally { try { @@ -829,7 +828,7 @@ public class SimplePostTool { * Appends to the path of the URL * @param url the URL * @param append the path to append - * @return the final URL version + * @return the final URL version */ protected static URL appendUrlPath(URL url, String append) throws MalformedURLException { return new URL(url.getProtocol() + "://" + url.getAuthority() + url.getPath() + append + (url.getQuery() != null ? "?"+url.getQuery() : "")); @@ -858,7 +857,7 @@ public class SimplePostTool { warn("The specified URL "+url+" is not a valid URL. Please check"); } } - + /** * Performs a simple get on the given URL */ @@ -919,7 +918,7 @@ public class SimplePostTool { } catch (IOException e) { fatal("IOException while posting data: " + e); } - + try { success &= checkResponseCode(urlc); try (final InputStream in = urlc.getInputStream()) { @@ -952,7 +951,7 @@ public class SimplePostTool { private static boolean checkResponseCode(HttpURLConnection urlc) throws IOException, GeneralSecurityException { if (urlc.getResponseCode() >= 400) { - warn("Solr returned an error #" + urlc.getResponseCode() + + warn("Solr returned an error #" + urlc.getResponseCode() + " (" + urlc.getResponseMessage() + ") for url: " + urlc.getURL()); Charset charset = StandardCharsets.ISO_8859_1; final String contentType = urlc.getContentType(); @@ -987,7 +986,7 @@ public class SimplePostTool { } /** - * Converts a string to an input stream + * Converts a string to an input stream * @param s the string * @return the input stream */ @@ -996,7 +995,7 @@ public class SimplePostTool { } /** - * Pipes everything from the source to the dest. If dest is null, + * Pipes everything from the source to the dest. If dest is null, * then everything is read from source and thrown away. */ private static void pipe(InputStream source, OutputStream dest) throws IOException { @@ -1020,7 +1019,7 @@ public class SimplePostTool { // // Utility methods for XPath handing // - + /** * Gets all nodes matching an XPath */ @@ -1030,7 +1029,7 @@ public class SimplePostTool { XPathExpression expr = xp.compile(xpath); return (NodeList) expr.evaluate(n, XPathConstants.NODESET); } - + /** * Gets the string content of the matching an XPath * @param n the node (or doc) @@ -1050,9 +1049,9 @@ public class SimplePostTool { } else return ""; } - + /** - * Takes a string as input and returns a DOM + * Takes a string as input and returns a DOM */ public static Document makeDom(byte[] in) throws SAXException, IOException, ParserConfigurationException { @@ -1069,7 +1068,7 @@ public class SimplePostTool { { private String _pattern; private Pattern p; - + public GlobFileFilter(String pattern, boolean isRegex) { _pattern = pattern; @@ -1085,32 +1084,32 @@ public class SimplePostTool { .replace("?", "."); _pattern = "^" + _pattern + "$"; } - + try { p = Pattern.compile(_pattern,Pattern.CASE_INSENSITIVE); } catch(PatternSyntaxException e) { fatal("Invalid type list "+pattern+". "+e.getDescription()); } } - + @Override public boolean accept(File file) { return p.matcher(file.getName()).find(); } } - + // // Simple crawler class which can fetch a page and check for robots.txt // class PageFetcher { Map> robotsCache; static final String DISALLOW = "Disallow:"; - + public PageFetcher() { robotsCache = new HashMap<>(); } - + public PageFetcherResult readPageFromUrl(URL u) { PageFetcherResult res = new PageFetcherResult(); try { @@ -1146,8 +1145,8 @@ public class SimplePostTool { } else { is = conn.getInputStream(); } - - // Read into memory, so that we later can pull links from the page without re-fetching + + // Read into memory, so that we later can pull links from the page without re-fetching res.content = inputStreamToByteArray(is); is.close(); } else { @@ -1160,7 +1159,7 @@ public class SimplePostTool { } return res; } - + public boolean isDisallowedByRobots(URL url) { String host = url.getHost(); String strRobot = url.getProtocol() + "://" + host + "/robots.txt"; @@ -1168,7 +1167,7 @@ public class SimplePostTool { if(disallows == null) { disallows = new ArrayList<>(); URL urlRobot; - try { + try { urlRobot = new URL(strRobot); disallows = parseRobotsTxt(urlRobot.openStream()); } catch (MalformedURLException e) { @@ -1177,7 +1176,7 @@ public class SimplePostTool { // There is no robots.txt, will cache an empty disallow list } } - + robotsCache.put(host, disallows); String strURL = url.getFile(); @@ -1254,7 +1253,7 @@ public class SimplePostTool { return l; } } - + /** * Utility class to hold the result form a page fetch */ diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java index b6ee3186842..7eddc766909 100755 --- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java +++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java @@ -152,15 +152,15 @@ import static org.apache.solr.common.params.CommonParams.NAME; /** * Command-line utility for working with Solr. */ -public class SolrCLI { +public class SolrCLI implements CLIO { private static final long MAX_WAIT_FOR_CORE_LOAD_NANOS = TimeUnit.NANOSECONDS.convert(1, TimeUnit.MINUTES); /** * Defines the interface to a Solr tool that can be run from this command-line app. */ public interface Tool { - String getName(); - Option[] getOptions(); + String getName(); + Option[] getOptions(); int runTool(CommandLine cli) throws Exception; } @@ -169,13 +169,13 @@ public class SolrCLI { protected boolean verbose = false; protected ToolBase() { - this(System.out); + this(CLIO.getOutStream()); } protected ToolBase(PrintStream stdout) { this.stdout = stdout; } - + protected void echoIfVerbose(final String msg, CommandLine cli) { if (cli.hasOption("verbose")) { echo(msg); @@ -196,7 +196,7 @@ public class SolrCLI { // since this is a CLI, spare the user the stacktrace String excMsg = exc.getMessage(); if (excMsg != null) { - System.err.println("\nERROR: " + excMsg + "\n"); + CLIO.err("\nERROR: " + excMsg + "\n"); toolExitStatus = 1; } else { throw exc; @@ -218,34 +218,34 @@ public class SolrCLI { public Option[] getOptions() { return cloudOptions; } - + protected void runImpl(CommandLine cli) throws Exception { raiseLogLevelUnlessVerbose(cli); String zkHost = cli.getOptionValue("zkHost", ZK_HOST); - + log.debug("Connecting to Solr cluster: " + zkHost); try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder(Collections.singletonList(zkHost), Optional.empty()).build()) { String collection = cli.getOptionValue("collection"); if (collection != null) cloudSolrClient.setDefaultCollection(collection); - + cloudSolrClient.connect(); runCloudTool(cloudSolrClient, cli); } } - + /** * Runs a SolrCloud tool with CloudSolrClient initialized */ protected abstract void runCloudTool(CloudSolrClient cloudSolrClient, CommandLine cli) throws Exception; } - + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - public static final String DEFAULT_SOLR_URL = "http://localhost:8983/solr"; + public static final String DEFAULT_SOLR_URL = "http://localhost:8983/solr"; public static final String ZK_HOST = "localhost:9983"; - + @SuppressWarnings("static-access") public static Option[] cloudOptions = new Option[] { OptionBuilder @@ -275,13 +275,13 @@ public class SolrCLI { throw new RuntimeException("SolrCLI failed to exit with status "+exitStatus); } } - + /** * Runs a tool. */ public static void main(String[] args) throws Exception { if (args == null || args.length == 0 || args[0] == null || args[0].trim().length() == 0) { - System.err.println("Invalid command-line args! Must pass the name of a tool to run.\n" + CLIO.err("Invalid command-line args! Must pass the name of a tool to run.\n" + "Supported tools:\n"); displayToolOptions(); exit(1); @@ -289,7 +289,7 @@ public class SolrCLI { if (args.length == 1 && Arrays.asList("-v","-version","version").contains(args[0])) { // Simple version tool, no need for its own class - System.out.println(Version.LATEST); + CLIO.out(Version.LATEST.toString()); exit(0); } @@ -320,7 +320,7 @@ public class SolrCLI { String[] toolArgs = toolArgList.toArray(new String[0]); // process command-line args to configure this application - CommandLine cli = + CommandLine cli = processCommandLineArgs(joinCommonAndToolOptions(toolOptions), toolArgs); List argList = cli.getArgList(); @@ -350,25 +350,25 @@ public class SolrCLI { if (keyStoreFile.isFile()) { System.setProperty(sysProp, keyStoreFile.getAbsolutePath()); } else { - System.err.println("WARNING: "+sysProp+" file "+keyStore+ + CLIO.err("WARNING: "+sysProp+" file "+keyStore+ " not found! https requests to Solr will likely fail; please update your "+ sysProp+" setting to use an absolute path."); } } - + private static void raiseLogLevelUnlessVerbose(CommandLine cli) { if (! cli.hasOption("verbose")) { StartupLoggingUtils.changeLogLevel("WARN"); } } - + /** * Support options common to all tools. */ public static Option[] getCommonToolOptions() { return new Option[0]; } - + // Creates an instance of the requested tool, using classpath scanning if necessary private static Tool newTool(String toolType) throws Exception { if ("healthcheck".equals(toolType)) @@ -418,12 +418,12 @@ public class SolrCLI { for (Class next : findToolClassesInPackage("org.apache.solr.util")) { Tool tool = next.newInstance(); if (toolType.equals(tool.getName())) - return tool; + return tool; } - + throw new IllegalArgumentException(toolType + " not supported!"); } - + private static void displayToolOptions() throws Exception { HelpFormatter formatter = new HelpFormatter(); formatter.printHelp("healthcheck", getToolOptions(new HealthcheckTool())); @@ -446,10 +446,10 @@ public class SolrCLI { List> toolClasses = findToolClassesInPackage("org.apache.solr.util"); for (Class next : toolClasses) { Tool tool = next.newInstance(); - formatter.printHelp(tool.getName(), getToolOptions(tool)); - } + formatter.printHelp(tool.getName(), getToolOptions(tool)); + } } - + private static Options getToolOptions(Tool tool) { Options options = new Options(); options.addOption("help", false, "Print this message"); @@ -459,41 +459,41 @@ public class SolrCLI { options.addOption(toolOpts[i]); return options; } - + public static Option[] joinCommonAndToolOptions(Option[] toolOpts) { return joinOptions(getCommonToolOptions(), toolOpts); } - + public static Option[] joinOptions(Option[] lhs, Option[] rhs) { List